From d3eed644e0cc1c52ac392bb8ae527ecc988a6762 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 12:52:31 +1000 Subject: [PATCH 01/27] feat(consumoor): bump default ClickHouse maxConns from 8 to 32 (#790) 78 table writers share a single connection pool. The old default of 8 was too low for concurrent flush bursts, causing unnecessary pool contention. 32 gives comfortable headroom without risking ClickHouse max_concurrent_queries limits across replicas. --- docs/consumoor-runbook.md | 4 ++-- example_consumoor.yaml | 2 +- pkg/consumoor/clickhouse/config.go | 2 +- pkg/consumoor/clickhouse/config_test.go | 2 +- pkg/consumoor/clickhouse/writer_test.go | 4 ++-- pkg/consumoor/config_test.go | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/consumoor-runbook.md b/docs/consumoor-runbook.md index 70afa2bc..46415776 100644 --- a/docs/consumoor-runbook.md +++ b/docs/consumoor-runbook.md @@ -265,7 +265,7 @@ Common causes of startup failure: ### Vertical Scaling -- Increase `maxConns` to allow more concurrent ClickHouse writes (default: 8) +- Increase `maxConns` to allow more concurrent ClickHouse writes (default: 32) - Increase `batchSize` to write more rows per INSERT (default: 200000) - Increase `bufferSize` to absorb more backpressure spikes (default: 200000), at the cost of higher memory usage - Increase pod CPU/memory to handle more concurrent table writers @@ -280,7 +280,7 @@ Common causes of startup failure: | `flushInterval` | `clickhouse.defaults.flushInterval` | 1s | Decrease for lower latency; increase if batches are too small | | `bufferSize` | `clickhouse.defaults.bufferSize` | 200000 | Increase to absorb ClickHouse hiccups without backpressure; decrease to limit memory | | `commitInterval` | `kafka.commitInterval` | 5s | Decrease to reduce duplicate replay window on crash; increase to reduce Kafka commit overhead | -| `maxConns` | `clickhouse.chgo.maxConns` | 8 | Increase when `chgo_pool_empty_acquire_total` is high | +| `maxConns` | `clickhouse.chgo.maxConns` | 32 | Increase when `chgo_pool_empty_acquire_total` is high | | `queryTimeout` | `clickhouse.chgo.queryTimeout` | 30s | Increase if large batches legitimately take longer to insert | | `maxRetries` | `clickhouse.chgo.maxRetries` | 3 | Increase if transient ClickHouse errors are frequent but recover quickly | | `deliveryMode` | `kafka.deliveryMode` | batch | Use `message` for safer per-message delivery; use `batch` for higher throughput | diff --git a/example_consumoor.yaml b/example_consumoor.yaml index 79db76be..27386be7 100644 --- a/example_consumoor.yaml +++ b/example_consumoor.yaml @@ -49,7 +49,7 @@ clickhouse: # maxRetries: 3 # retryBaseDelay: 100ms # retryMaxDelay: 2s - # maxConns: 8 + # maxConns: 32 # minConns: 1 # connMaxLifetime: 1h # connMaxIdleTime: 10m diff --git a/pkg/consumoor/clickhouse/config.go b/pkg/consumoor/clickhouse/config.go index 75ae8296..5d452a73 100644 --- a/pkg/consumoor/clickhouse/config.go +++ b/pkg/consumoor/clickhouse/config.go @@ -98,7 +98,7 @@ type ChGoConfig struct { RetryMaxDelay time.Duration `yaml:"retryMaxDelay" default:"2s"` // MaxConns is the maximum number of pooled ClickHouse connections. - MaxConns int32 `yaml:"maxConns" default:"8"` + MaxConns int32 `yaml:"maxConns" default:"32"` // MinConns is the minimum number of pooled ClickHouse connections. MinConns int32 `yaml:"minConns" default:"1"` // ConnMaxLifetime is the maximum lifetime for pooled connections. diff --git a/pkg/consumoor/clickhouse/config_test.go b/pkg/consumoor/clickhouse/config_test.go index 5a1a9e0b..6763edd7 100644 --- a/pkg/consumoor/clickhouse/config_test.go +++ b/pkg/consumoor/clickhouse/config_test.go @@ -31,7 +31,7 @@ func validChGoConfig() ChGoConfig { MaxRetries: 3, RetryBaseDelay: 100 * time.Millisecond, RetryMaxDelay: 2 * time.Second, - MaxConns: 8, + MaxConns: 32, MinConns: 1, ConnMaxLifetime: 1 * time.Hour, ConnMaxIdleTime: 10 * time.Minute, diff --git a/pkg/consumoor/clickhouse/writer_test.go b/pkg/consumoor/clickhouse/writer_test.go index 34c3410c..4f074bfe 100644 --- a/pkg/consumoor/clickhouse/writer_test.go +++ b/pkg/consumoor/clickhouse/writer_test.go @@ -85,7 +85,7 @@ func TestChGoConfigValidateAcceptsValidConfig(t *testing.T) { ReadTimeout: 30 * time.Second, RetryBaseDelay: 100 * time.Millisecond, RetryMaxDelay: 2 * time.Second, - MaxConns: 8, + MaxConns: 32, MinConns: 0, ConnMaxLifetime: time.Hour, ConnMaxIdleTime: 10 * time.Minute, @@ -421,7 +421,7 @@ func TestBufferWarningThresholdValidation(t *testing.T) { ReadTimeout: 30 * time.Second, RetryBaseDelay: 100 * time.Millisecond, RetryMaxDelay: 2 * time.Second, - MaxConns: 8, + MaxConns: 32, MinConns: 1, ConnMaxLifetime: time.Hour, ConnMaxIdleTime: 10 * time.Minute, diff --git a/pkg/consumoor/config_test.go b/pkg/consumoor/config_test.go index f2d9ebf2..66b664bd 100644 --- a/pkg/consumoor/config_test.go +++ b/pkg/consumoor/config_test.go @@ -20,7 +20,7 @@ func validChGoConfig() clickhouse.ChGoConfig { MaxRetries: 3, RetryBaseDelay: 100 * time.Millisecond, RetryMaxDelay: 2 * time.Second, - MaxConns: 8, + MaxConns: 32, MinConns: 1, ConnMaxLifetime: time.Hour, ConnMaxIdleTime: 10 * time.Minute, From 49cf6c8dca3c334c9a33be7d4cafef9a8bc9d5b1 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 13:25:11 +1000 Subject: [PATCH 02/27] feat(consumoor): add per-table adaptive concurrency limiting for ClickHouse INSERTs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds optional AIMD-based adaptive concurrency control using failsafe-go's AdaptiveLimiter. Each table writer gets its own independent limiter that adjusts INSERT concurrency based on observed ClickHouse latency. Disabled by default (zero behavior change). When enabled via chgo.adaptiveLimiter.enabled, each table independently ramps up or backs off. The shared chpool.Pool MaxConns acts as the global ceiling — if all tables ramp up simultaneously, pool contention rises and each limiter self-corrects. Limiter rejection is a distinct third error category (not retryable, not permanent): the table writer sets flushBlocked but does not engage organic retry backoff, waiting for the next normal ticker cycle instead. --- go.mod | 4 +- go.sum | 12 +- pkg/consumoor/clickhouse/adaptive_limiter.go | 79 +++++++++++++ .../clickhouse/adaptive_limiter_test.go | 108 +++++++++++++++++ pkg/consumoor/clickhouse/config.go | 69 +++++++++++ pkg/consumoor/clickhouse/config_test.go | 110 ++++++++++++++++++ pkg/consumoor/clickhouse/errors.go | 21 ++++ pkg/consumoor/clickhouse/pool_metrics.go | 17 +++ pkg/consumoor/clickhouse/table_writer.go | 28 ++++- pkg/consumoor/clickhouse/writer.go | 5 + pkg/consumoor/clickhouse/writer_test.go | 16 +++ pkg/consumoor/telemetry/metrics.go | 30 +++++ 12 files changed, 495 insertions(+), 4 deletions(-) create mode 100644 pkg/consumoor/clickhouse/adaptive_limiter.go create mode 100644 pkg/consumoor/clickhouse/adaptive_limiter_test.go diff --git a/go.mod b/go.mod index e18045b8..746a694e 100644 --- a/go.mod +++ b/go.mod @@ -21,6 +21,7 @@ require ( github.com/ethpandaops/beacon v0.65.0 github.com/ethpandaops/ethcore v0.0.0-20260112064422-e7fe02956738 github.com/ethpandaops/ethwallclock v0.4.0 + github.com/failsafe-go/failsafe-go v0.9.6 github.com/ferranbt/fastssz v1.0.0 github.com/go-co-op/gocron/v2 v2.16.6 github.com/golang/protobuf v1.5.4 @@ -83,7 +84,7 @@ require ( github.com/VictoriaMetrics/fastcache v1.13.0 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/bits-and-blooms/bitset v1.24.0 // indirect + github.com/bits-and-blooms/bitset v1.24.4 // indirect github.com/btcsuite/btcd/btcec/v2 v2.3.5 // indirect github.com/cenkalti/backoff/v4 v4.3.0 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect @@ -155,6 +156,7 @@ require ( github.com/huin/goupnp v1.3.0 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/influxdata/go-syslog/v3 v3.0.0 // indirect + github.com/influxdata/tdigest v0.0.1 // indirect github.com/ipfs/go-cid v0.5.0 // indirect github.com/ipfs/go-log/v2 v2.8.0 // indirect github.com/itchyny/gojq v0.12.18 // indirect diff --git a/go.sum b/go.sum index ed044eee..1f23646a 100644 --- a/go.sum +++ b/go.sum @@ -63,8 +63,8 @@ github.com/benbjohnson/clock v1.3.5 h1:VvXlSJBzZpA/zum6Sj74hxwYI2DIxRWuNIoXAzHZz github.com/benbjohnson/clock v1.3.5/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bits-and-blooms/bitset v1.24.0 h1:H4x4TuulnokZKvHLfzVRTHJfFfnHEeSYJizujEZvmAM= -github.com/bits-and-blooms/bitset v1.24.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= +github.com/bits-and-blooms/bitset v1.24.4 h1:95H15Og1clikBrKr/DuzMXkQzECs1M6hhoGXLwLQOZE= +github.com/bits-and-blooms/bitset v1.24.4/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= github.com/bsm/ginkgo/v2 v2.12.0 h1:Ny8MWAHyOepLGlLKYmXG4IEkioBysk6GpaRTLC8zwWs= github.com/bsm/ginkgo/v2 v2.12.0/go.mod h1:SwYbGRRDovPVboqFv0tPTcG1sN61LM1Z4ARdbAV9g4c= github.com/bsm/gomega v1.27.10 h1:yeMWxP2pV2fG3FgAODIY8EiRE3dy0aeFYt4l7wh6yKA= @@ -193,6 +193,8 @@ github.com/ethpandaops/ethereum-package-go v0.8.0 h1:h4hyCTBgW+fWNxLLLP29GAhk0BM github.com/ethpandaops/ethereum-package-go v0.8.0/go.mod h1:LQThCwlCeeNBTdXOFV+xSwudoced53x+o/Orya3Y+oo= github.com/ethpandaops/ethwallclock v0.4.0 h1:+sgnhf4pk6hLPukP076VxkiLloE4L0Yk1yat+ZyHh1g= github.com/ethpandaops/ethwallclock v0.4.0/go.mod h1:y0Cu+mhGLlem19vnAV2x0hpFS5KZ7oOi2SWYayv9l24= +github.com/failsafe-go/failsafe-go v0.9.6 h1:vPSH2cry0Ee5cnR9wc9qshCDO6jdrMA9elBJNwyo4Uk= +github.com/failsafe-go/failsafe-go v0.9.6/go.mod h1:IeRpglkcwzKagjDMh90ZhN2l4Ovt3+jemQBUbThag54= github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= @@ -353,6 +355,8 @@ github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2 github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/influxdata/go-syslog/v3 v3.0.0 h1:jichmjSZlYK0VMmlz+k4WeOQd7z745YLsvGMqwtYt4I= github.com/influxdata/go-syslog/v3 v3.0.0/go.mod h1:tulsOp+CecTAYC27u9miMgq21GqXRW6VdKbOG+QSP4Q= +github.com/influxdata/tdigest v0.0.1 h1:XpFptwYmnEKUqmkcDjrzffswZ3nvNeevbUSLPP/ZzIY= +github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= github.com/ipfs/go-cid v0.5.0 h1:goEKKhaGm0ul11IHA7I6p1GmKz8kEYniqFopaB5Otwg= github.com/ipfs/go-cid v0.5.0/go.mod h1:0L7vmeNXpQpUS9vt+yEARkJ8rOg43DF3iPgn4GIN0mk= github.com/ipfs/go-log/v2 v2.8.0 h1:SptNTPJQV3s5EF4FdrTu/yVdOKfGbDgn1EBZx4til2o= @@ -878,6 +882,7 @@ golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98y golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/crypto v0.47.0 h1:V6e3FRj+n4dbpw86FJ8Fv7XVOql7TEwpHapKoMJ/GO8= golang.org/x/crypto v0.47.0/go.mod h1:ff3Y9VzzKbwSSEzWqJsJVBnWmRwRSHt/6Op5n9bQc4A= +golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20251002181428-27f1f14c8bb9 h1:TQwNpfvNkxAVlItJf6Cr5JTsVZoC/Sj7K3OZv2Pc14A= golang.org/x/exp v0.0.0-20251002181428-27f1f14c8bb9/go.mod h1:TwQYMMnGpvZyc+JpB/UAuTNIsVJifOlSkrZkhcvpVUk= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -977,6 +982,7 @@ golang.org/x/text v0.33.0 h1:B3njUFyqtHDUI5jMn1YIr5B0IE2U0qck04r6d4KPAxE= golang.org/x/text v0.33.0/go.mod h1:LuMebE6+rBincTi9+xWTY8TztLzKHc/9C1uBCG27+q8= golang.org/x/time v0.13.0 h1:eUlYslOIt32DgYD6utsuUeHs4d7AsEYLuIAdg7FlYgI= golang.org/x/time v0.13.0/go.mod h1:eL/Oa2bBBK0TkX57Fyni+NgnyQQN4LitPmob2Hjnqw4= +golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= @@ -991,8 +997,10 @@ golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20220517211312-f3a8303e98df/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= +gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.16.0 h1:5+ul4Swaf3ESvrOnidPp4GZbzf0mxVQpDCYUQE7OJfk= gonum.org/v1/gonum v0.16.0/go.mod h1:fef3am4MQ93R2HHpKnLk4/Tbh/s0+wqD5nfa6Pnwy4E= +gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= google.golang.org/genproto/googleapis/api v0.0.0-20260128011058-8636f8732409 h1:merA0rdPeUV3YIIfHHcH4qBkiQAc1nfCKSI7lB4cV2M= google.golang.org/genproto/googleapis/api v0.0.0-20260128011058-8636f8732409/go.mod h1:fl8J1IvUjCilwZzQowmw2b7HQB2eAuYBabMXzWurF+I= google.golang.org/genproto/googleapis/rpc v0.0.0-20260128011058-8636f8732409 h1:H86B94AW+VfJWDqFeEbBPhEtHzJwJfTbgE2lZa54ZAQ= diff --git a/pkg/consumoor/clickhouse/adaptive_limiter.go b/pkg/consumoor/clickhouse/adaptive_limiter.go new file mode 100644 index 00000000..176e3b49 --- /dev/null +++ b/pkg/consumoor/clickhouse/adaptive_limiter.go @@ -0,0 +1,79 @@ +package clickhouse + +import ( + "context" + + "github.com/failsafe-go/failsafe-go/adaptivelimiter" +) + +// adaptiveConcurrencyLimiter wraps failsafe-go's AdaptiveLimiter to provide +// per-table AIMD-based concurrency control around ClickHouse INSERTs. +type adaptiveConcurrencyLimiter struct { + limiter adaptivelimiter.AdaptiveLimiter[any] +} + +// newAdaptiveConcurrencyLimiter creates a limiter from config. Returns nil +// when disabled, which callers treat as a no-op passthrough. +func newAdaptiveConcurrencyLimiter( + cfg AdaptiveLimiterConfig, +) *adaptiveConcurrencyLimiter { + if !cfg.Enabled { + return nil + } + + builder := adaptivelimiter.NewBuilder[any](). + WithLimits(cfg.MinLimit, cfg.MaxLimit, cfg.InitialLimit) + + if cfg.QueueInitialRejectionFactor > 0 && cfg.QueueMaxRejectionFactor > 0 { + builder = builder.WithQueueing( + cfg.QueueInitialRejectionFactor, + cfg.QueueMaxRejectionFactor, + ) + } + + return &adaptiveConcurrencyLimiter{ + limiter: builder.Build(), + } +} + +// Limit returns the current concurrency limit. +func (l *adaptiveConcurrencyLimiter) Limit() int { + return l.limiter.Limit() +} + +// Inflight returns the number of in-flight permits. +func (l *adaptiveConcurrencyLimiter) Inflight() int { + return l.limiter.Inflight() +} + +// Queued returns the number of queued permit requests. +func (l *adaptiveConcurrencyLimiter) Queued() int { + return l.limiter.Queued() +} + +// doWithLimiter acquires a permit, executes fn, and records the outcome. +// On success the permit is recorded (RTT measured); on failure the permit +// is dropped (signals the limiter to decrease concurrency). +// Returns limiterRejectedError if the limiter rejects the request. +func (l *adaptiveConcurrencyLimiter) doWithLimiter( + ctx context.Context, + fn func(context.Context) error, +) error { + permit, ok := l.limiter.TryAcquirePermit() + if !ok { + return &limiterRejectedError{ + cause: adaptivelimiter.ErrExceeded, + } + } + + err := fn(ctx) + if err != nil { + permit.Drop() + + return err + } + + permit.Record() + + return nil +} diff --git a/pkg/consumoor/clickhouse/adaptive_limiter_test.go b/pkg/consumoor/clickhouse/adaptive_limiter_test.go new file mode 100644 index 00000000..ebe48f39 --- /dev/null +++ b/pkg/consumoor/clickhouse/adaptive_limiter_test.go @@ -0,0 +1,108 @@ +package clickhouse + +import ( + "context" + "errors" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestNewAdaptiveConcurrencyLimiter_DisabledReturnsNil(t *testing.T) { + limiter := newAdaptiveConcurrencyLimiter(AdaptiveLimiterConfig{ + Enabled: false, + }) + + assert.Nil(t, limiter) +} + +func TestNewAdaptiveConcurrencyLimiter_EnabledReturnsLimiter(t *testing.T) { + limiter := newAdaptiveConcurrencyLimiter(AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 50, + InitialLimit: 8, + QueueInitialRejectionFactor: 2, + QueueMaxRejectionFactor: 3, + }) + + require.NotNil(t, limiter) + assert.Equal(t, 8, limiter.Limit()) + assert.Equal(t, 0, limiter.Inflight()) + assert.Equal(t, 0, limiter.Queued()) +} + +func TestAdaptiveConcurrencyLimiter_AcquireAndRecord(t *testing.T) { + limiter := newAdaptiveConcurrencyLimiter(AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 50, + InitialLimit: 8, + }) + + require.NotNil(t, limiter) + + err := limiter.doWithLimiter(context.Background(), func(_ context.Context) error { + // While in-flight, inflight should be >= 1. + assert.GreaterOrEqual(t, limiter.Inflight(), 1) + + return nil + }) + + require.NoError(t, err) +} + +func TestAdaptiveConcurrencyLimiter_AcquireAndDropOnError(t *testing.T) { + limiter := newAdaptiveConcurrencyLimiter(AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 50, + InitialLimit: 8, + }) + + require.NotNil(t, limiter) + + sentinel := errors.New("simulated failure") + + err := limiter.doWithLimiter(context.Background(), func(_ context.Context) error { + return sentinel + }) + + require.ErrorIs(t, err, sentinel) + // After drop, inflight should return to 0. + assert.Equal(t, 0, limiter.Inflight()) +} + +func TestAdaptiveConcurrencyLimiter_RejectsWhenFull(t *testing.T) { + // Create a limiter with limit=1 so we can saturate it. + limiter := newAdaptiveConcurrencyLimiter(AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 1, + InitialLimit: 1, + }) + + require.NotNil(t, limiter) + + // Acquire the single permit. + permit, ok := limiter.limiter.TryAcquirePermit() + require.True(t, ok, "should acquire the single permit") + + // Second attempt should be rejected. + err := limiter.doWithLimiter(context.Background(), func(_ context.Context) error { + return nil + }) + + require.Error(t, err) + assert.True(t, IsLimiterRejected(err)) + + // Release and verify we can acquire again. + permit.Record() + + err = limiter.doWithLimiter(context.Background(), func(_ context.Context) error { + return nil + }) + + require.NoError(t, err) +} diff --git a/pkg/consumoor/clickhouse/config.go b/pkg/consumoor/clickhouse/config.go index 5d452a73..4361799d 100644 --- a/pkg/consumoor/clickhouse/config.go +++ b/pkg/consumoor/clickhouse/config.go @@ -79,6 +79,68 @@ type TableConfig struct { InsertSettings map[string]any `yaml:"insertSettings"` } +// AdaptiveLimiterConfig configures per-table adaptive concurrency limiting. +// When enabled, each table writer independently adjusts its INSERT concurrency +// based on observed ClickHouse latency using an AIMD algorithm. +type AdaptiveLimiterConfig struct { + // Enabled turns on adaptive concurrency limiting. Disabled by default + // for zero behavior change unless explicitly opted in. + Enabled bool `yaml:"enabled" default:"false"` + // MinLimit is the minimum concurrent INSERTs the limiter allows. + MinLimit uint `yaml:"minLimit" default:"1"` + // MaxLimit caps the maximum concurrent INSERTs the limiter allows. + MaxLimit uint `yaml:"maxLimit" default:"50"` + // InitialLimit is the starting concurrency before adaptation. + InitialLimit uint `yaml:"initialLimit" default:"8"` + // QueueInitialRejectionFactor controls the queue size below which + // requests are rejected during the initial learning phase. + QueueInitialRejectionFactor float64 `yaml:"queueInitialRejectionFactor" default:"2"` + // QueueMaxRejectionFactor controls the queue size below which + // requests are rejected after the initial learning phase. + QueueMaxRejectionFactor float64 `yaml:"queueMaxRejectionFactor" default:"3"` +} + +// Validate checks the adaptive limiter configuration for errors. +func (c *AdaptiveLimiterConfig) Validate() error { + if !c.Enabled { + return nil + } + + if c.MinLimit == 0 { + return errors.New("clickhouse.chgo.adaptiveLimiter: minLimit must be > 0") + } + + if c.MaxLimit == 0 { + return errors.New("clickhouse.chgo.adaptiveLimiter: maxLimit must be > 0") + } + + if c.MinLimit > c.MaxLimit { + return errors.New( + "clickhouse.chgo.adaptiveLimiter: minLimit must be <= maxLimit", + ) + } + + if c.InitialLimit < c.MinLimit || c.InitialLimit > c.MaxLimit { + return errors.New( + "clickhouse.chgo.adaptiveLimiter: initialLimit must be between minLimit and maxLimit", + ) + } + + if c.QueueInitialRejectionFactor <= 0 { + return errors.New( + "clickhouse.chgo.adaptiveLimiter: queueInitialRejectionFactor must be > 0", + ) + } + + if c.QueueMaxRejectionFactor <= 0 { + return errors.New( + "clickhouse.chgo.adaptiveLimiter: queueMaxRejectionFactor must be > 0", + ) + } + + return nil +} + // ChGoConfig configures the ch-go backend query retries and connection pooling. type ChGoConfig struct { // DialTimeout is the timeout for establishing a connection to ClickHouse. @@ -111,6 +173,9 @@ type ChGoConfig struct { // PoolMetricsInterval controls how often pool stats are sampled. // Set to 0 to disable pool metrics collection. PoolMetricsInterval time.Duration `yaml:"poolMetricsInterval" default:"15s"` + + // AdaptiveLimiter configures per-table adaptive concurrency limiting. + AdaptiveLimiter AdaptiveLimiterConfig `yaml:"adaptiveLimiter"` } // Validate checks the ClickHouse configuration for errors. @@ -245,6 +310,10 @@ func (c *ChGoConfig) Validate() error { return errors.New("clickhouse.chgo: poolMetricsInterval must be >= 0") } + if err := c.AdaptiveLimiter.Validate(); err != nil { + return err + } + return nil } diff --git a/pkg/consumoor/clickhouse/config_test.go b/pkg/consumoor/clickhouse/config_test.go index 6763edd7..55110637 100644 --- a/pkg/consumoor/clickhouse/config_test.go +++ b/pkg/consumoor/clickhouse/config_test.go @@ -175,3 +175,113 @@ func TestConfig_Validate_OrganicRetry(t *testing.T) { }) } } + +func TestAdaptiveLimiterConfig_Validate(t *testing.T) { + tests := []struct { + name string + cfg AdaptiveLimiterConfig + wantErr string + }{ + { + name: "disabled passes", + cfg: AdaptiveLimiterConfig{Enabled: false}, + }, + { + name: "valid enabled", + cfg: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 50, + InitialLimit: 8, + QueueInitialRejectionFactor: 2, + QueueMaxRejectionFactor: 3, + }, + }, + { + name: "zero minLimit", + cfg: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 0, + MaxLimit: 50, + InitialLimit: 8, + }, + wantErr: "minLimit must be > 0", + }, + { + name: "zero maxLimit", + cfg: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 0, + InitialLimit: 1, + }, + wantErr: "maxLimit must be > 0", + }, + { + name: "minLimit exceeds maxLimit", + cfg: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 10, + MaxLimit: 5, + InitialLimit: 5, + }, + wantErr: "minLimit must be <= maxLimit", + }, + { + name: "initialLimit below minLimit", + cfg: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 5, + MaxLimit: 50, + InitialLimit: 2, + }, + wantErr: "initialLimit must be between minLimit and maxLimit", + }, + { + name: "initialLimit above maxLimit", + cfg: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 10, + InitialLimit: 20, + }, + wantErr: "initialLimit must be between minLimit and maxLimit", + }, + { + name: "zero queueInitialRejectionFactor", + cfg: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 50, + InitialLimit: 8, + QueueInitialRejectionFactor: 0, + QueueMaxRejectionFactor: 3, + }, + wantErr: "queueInitialRejectionFactor must be > 0", + }, + { + name: "zero queueMaxRejectionFactor", + cfg: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 50, + InitialLimit: 8, + QueueInitialRejectionFactor: 2, + QueueMaxRejectionFactor: 0, + }, + wantErr: "queueMaxRejectionFactor must be > 0", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := tt.cfg.Validate() + if tt.wantErr != "" { + require.Error(t, err) + assert.Contains(t, err.Error(), tt.wantErr) + } else { + require.NoError(t, err) + } + }) + } +} diff --git a/pkg/consumoor/clickhouse/errors.go b/pkg/consumoor/clickhouse/errors.go index 2c6bd71a..1192f26c 100644 --- a/pkg/consumoor/clickhouse/errors.go +++ b/pkg/consumoor/clickhouse/errors.go @@ -45,6 +45,27 @@ type flattenError struct { func (e *flattenError) Error() string { return fmt.Sprintf("flatten failed: %v", e.cause) } func (e *flattenError) Unwrap() error { return e.cause } +// limiterRejectedError indicates the adaptive concurrency limiter rejected +// the request. This is not retryable (doWithRetry exits immediately) and not +// permanent — the table writer simply waits for the next flush cycle. +type limiterRejectedError struct { + cause error +} + +func (e *limiterRejectedError) Error() string { + return fmt.Sprintf("adaptive limiter rejected: %v", e.cause) +} + +func (e *limiterRejectedError) Unwrap() error { return e.cause } + +// IsLimiterRejected reports whether err was caused by adaptive concurrency +// limiter rejection. +func IsLimiterRejected(err error) bool { + var lre *limiterRejectedError + + return errors.As(err, &lre) +} + // IsPermanentWriteError returns true for errors that will never succeed on // retry: schema mismatches, type errors, conversion failures. func IsPermanentWriteError(err error) bool { diff --git a/pkg/consumoor/clickhouse/pool_metrics.go b/pkg/consumoor/clickhouse/pool_metrics.go index 7c7ef4d2..39f942a4 100644 --- a/pkg/consumoor/clickhouse/pool_metrics.go +++ b/pkg/consumoor/clickhouse/pool_metrics.go @@ -52,6 +52,23 @@ func (w *ChGoWriter) collectPoolMetrics() { } prevCanceledAcquireCount = canceledAcquireCount + + w.collectAdaptiveLimiterMetrics() } } } + +func (w *ChGoWriter) collectAdaptiveLimiterMetrics() { + w.mu.RLock() + defer w.mu.RUnlock() + + for _, tw := range w.tables { + if tw.limiter == nil { + continue + } + + w.metrics.AdaptiveLimiterLimit().WithLabelValues(tw.table).Set(float64(tw.limiter.Limit())) + w.metrics.AdaptiveLimiterInflight().WithLabelValues(tw.table).Set(float64(tw.limiter.Inflight())) + w.metrics.AdaptiveLimiterQueued().WithLabelValues(tw.table).Set(float64(tw.limiter.Queued())) + } +} diff --git a/pkg/consumoor/clickhouse/table_writer.go b/pkg/consumoor/clickhouse/table_writer.go index c9a94c93..9bad6c6f 100644 --- a/pkg/consumoor/clickhouse/table_writer.go +++ b/pkg/consumoor/clickhouse/table_writer.go @@ -40,6 +40,10 @@ type chTableWriter struct { newBatch func() route.ColumnarBatch batch route.ColumnarBatch // reused across flushes via Reset() + // limiter is the per-table adaptive concurrency limiter. + // nil when adaptive limiting is disabled. + limiter *adaptiveConcurrencyLimiter + // lastWarnAt tracks the last time a buffer warning was emitted (Unix nanos). // Accessed atomically from the Write goroutine for rate limiting. lastWarnAt atomic.Int64 @@ -85,6 +89,12 @@ func (tw *chTableWriter) run(done <-chan struct{}) { continue } + if IsLimiterRejected(err) { + flushBlocked = true + + continue + } + // Preserve events for retry on next flush cycle. While // pending, stop draining tw.buffer so channel backpressure // remains bounded by BufferSize. @@ -123,6 +133,12 @@ func (tw *chTableWriter) run(done <-chan struct{}) { continue } + if IsLimiterRejected(err) { + flushBlocked = true + + continue + } + flushBlocked = true tw.scheduleOrganicRetry(ticker, retryAttempt) @@ -174,6 +190,8 @@ func (tw *chTableWriter) run(done <-chan struct{}) { flushBlocked = false tw.resetRetryState(ticker, &retryAttempt) + case IsLimiterRejected(err): + flushBlocked = true default: flushBlocked = true } @@ -368,7 +386,7 @@ func (tw *chTableWriter) do( query *ch.Query, beforeAttempt func(), ) error { - return tw.writer.doWithRetry(ctx, operation, func(attemptCtx context.Context) error { + poolFn := func(attemptCtx context.Context) error { if beforeAttempt != nil { beforeAttempt() } @@ -383,6 +401,14 @@ func (tw *chTableWriter) do( } return pool.Do(attemptCtx, *query) + } + + return tw.writer.doWithRetry(ctx, operation, func(attemptCtx context.Context) error { + if tw.limiter == nil { + return poolFn(attemptCtx) + } + + return tw.limiter.doWithLimiter(attemptCtx, poolFn) }) } diff --git a/pkg/consumoor/clickhouse/writer.go b/pkg/consumoor/clickhouse/writer.go index 990df570..48cf8522 100644 --- a/pkg/consumoor/clickhouse/writer.go +++ b/pkg/consumoor/clickhouse/writer.go @@ -304,6 +304,7 @@ func (w *ChGoWriter) getOrCreateTableWriter(table string) *chTableWriter { organicRetryMaxDelay: w.config.OrganicRetryMaxDelay, drainTimeout: w.config.DrainTimeout, newBatch: w.batchFactories[table], + limiter: newAdaptiveConcurrencyLimiter(w.chgoCfg.AdaptiveLimiter), } w.tables[writeTable] = tw @@ -387,6 +388,10 @@ func (w *ChGoWriter) doWithRetry( lastErr = err + if IsLimiterRejected(err) { + return err + } + if !isRetryableError(err) { return err } diff --git a/pkg/consumoor/clickhouse/writer_test.go b/pkg/consumoor/clickhouse/writer_test.go index 4f074bfe..98c2dab3 100644 --- a/pkg/consumoor/clickhouse/writer_test.go +++ b/pkg/consumoor/clickhouse/writer_test.go @@ -301,6 +301,22 @@ func TestDoWithRetry_NonRetryableExitsImmediately(t *testing.T) { assert.Equal(t, int32(1), calls.Load()) } +func TestDoWithRetry_LimiterRejectionExitsImmediately(t *testing.T) { + w := newTestWriter(5, time.Millisecond, 10*time.Millisecond) + + var calls atomic.Int32 + + err := w.doWithRetry(context.Background(), "test_op", func(_ context.Context) error { + calls.Add(1) + + return &limiterRejectedError{cause: errors.New("limit exceeded")} + }) + + require.Error(t, err) + assert.True(t, IsLimiterRejected(err), "error should be classified as limiter rejected") + assert.Equal(t, int32(1), calls.Load(), "should not retry on limiter rejection") +} + func TestDoWithRetry_BackoffTiming(t *testing.T) { baseDelay := 20 * time.Millisecond maxDelay := 100 * time.Millisecond diff --git a/pkg/consumoor/telemetry/metrics.go b/pkg/consumoor/telemetry/metrics.go index bd08af76..ba8a61d5 100644 --- a/pkg/consumoor/telemetry/metrics.go +++ b/pkg/consumoor/telemetry/metrics.go @@ -24,6 +24,11 @@ type Metrics struct { activeTopics prometheus.Gauge kafkaConsumerLag *prometheus.GaugeVec + // adaptive limiter metrics (per-table) + adaptiveLimiterLimit *prometheus.GaugeVec + adaptiveLimiterInflight *prometheus.GaugeVec + adaptiveLimiterQueued *prometheus.GaugeVec + // ch-go pool metrics chgoPoolAcquiredResources prometheus.Gauge chgoPoolIdleResources prometheus.Gauge @@ -160,6 +165,27 @@ func NewMetrics(namespace string) *Metrics { Help: "Kafka consumer group lag per topic and partition.", }, []string{"topic", "partition", "consumer_group"}), + adaptiveLimiterLimit: promauto.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "adaptive_limiter_limit", + Help: "Current adaptive concurrency limit per table.", + }, []string{"table"}), + + adaptiveLimiterInflight: promauto.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "adaptive_limiter_inflight", + Help: "Current number of in-flight permits per table.", + }, []string{"table"}), + + adaptiveLimiterQueued: promauto.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "adaptive_limiter_queued", + Help: "Current number of queued permit requests per table.", + }, []string{"table"}), + chgoPoolAcquiredResources: promauto.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, @@ -249,6 +275,10 @@ func (m *Metrics) FlattenErrors() *prometheus.CounterVec { return m.flattenEr func (m *Metrics) ActiveTopics() prometheus.Gauge { return m.activeTopics } func (m *Metrics) KafkaConsumerLag() *prometheus.GaugeVec { return m.kafkaConsumerLag } +func (m *Metrics) AdaptiveLimiterLimit() *prometheus.GaugeVec { return m.adaptiveLimiterLimit } +func (m *Metrics) AdaptiveLimiterInflight() *prometheus.GaugeVec { return m.adaptiveLimiterInflight } +func (m *Metrics) AdaptiveLimiterQueued() *prometheus.GaugeVec { return m.adaptiveLimiterQueued } + func (m *Metrics) ChgoPoolAcquiredResources() prometheus.Gauge { return m.chgoPoolAcquiredResources } func (m *Metrics) ChgoPoolIdleResources() prometheus.Gauge { return m.chgoPoolIdleResources } func (m *Metrics) ChgoPoolConstructingResources() prometheus.Gauge { From 36a903d721e4750b85754a5012292e87527da1d6 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 14:03:03 +1000 Subject: [PATCH 03/27] feat(consumoor): add configurable TCP connect timeout for kafka brokers (#791) Adds a `connectTimeout` config field (default 10s) that sets `tcp.connect_timeout` on the kafka_franz input. This prevents hung TCP dials from generating noisy "is SASL missing?" warnings when some brokers are temporarily unreachable. --- pkg/consumoor/source/benthos.go | 6 +++++ pkg/consumoor/source/benthos_test.go | 36 ++++++++++++++++++++++++++++ pkg/consumoor/source/config.go | 5 ++++ 3 files changed, 47 insertions(+) diff --git a/pkg/consumoor/source/benthos.go b/pkg/consumoor/source/benthos.go index 55fde549..27b87dd0 100644 --- a/pkg/consumoor/source/benthos.go +++ b/pkg/consumoor/source/benthos.go @@ -114,6 +114,12 @@ func benthosConfigYAML(logLevel string, kafkaConfig *KafkaConfig) ([]byte, error inputKafka["metadata_max_age"] = kafkaConfig.TopicRefreshInterval.String() } + if kafkaConfig.ConnectTimeout > 0 { + inputKafka["tcp"] = map[string]any{ + "connect_timeout": kafkaConfig.ConnectTimeout.String(), + } + } + if kafkaConfig.TLS.Enabled { tlsObj := map[string]any{ "enabled": true, diff --git a/pkg/consumoor/source/benthos_test.go b/pkg/consumoor/source/benthos_test.go index b3a208e2..b81aa2a2 100644 --- a/pkg/consumoor/source/benthos_test.go +++ b/pkg/consumoor/source/benthos_test.go @@ -148,6 +148,7 @@ func TestBenthosConfigYAML(t *testing.T) { SessionTimeoutMs: 30000, OffsetDefault: "latest", CommitInterval: 7 * time.Second, + ConnectTimeout: 10 * time.Second, } raw, err := benthosConfigYAML("debug", cfg) @@ -166,6 +167,10 @@ func TestBenthosConfigYAML(t *testing.T) { assert.Equal(t, "latest", kafka["start_offset"]) assert.Equal(t, "7s", kafka["commit_period"]) + tcp, ok := kafka["tcp"].(map[string]any) + require.True(t, ok, "tcp block should be present when ConnectTimeout > 0") + assert.Equal(t, "10s", tcp["connect_timeout"]) + output, ok := parsed["output"].(map[string]any) require.True(t, ok) @@ -173,6 +178,37 @@ func TestBenthosConfigYAML(t *testing.T) { assert.True(t, hasOutput) } +func TestBenthosConfigYAML_NoTCPBlockWhenConnectTimeoutZero(t *testing.T) { + cfg := &KafkaConfig{ + Brokers: []string{"kafka-1:9092"}, + Topics: []string{"^test-.+"}, + ConsumerGroup: "xatu-consumoor", + Encoding: "json", + FetchMinBytes: 1, + FetchWaitMaxMs: 250, + MaxPartitionFetchBytes: 1048576, + SessionTimeoutMs: 30000, + OffsetDefault: "earliest", + CommitInterval: 5 * time.Second, + ConnectTimeout: 0, + } + + raw, err := benthosConfigYAML("info", cfg) + require.NoError(t, err) + + var parsed map[string]any + require.NoError(t, yaml.Unmarshal(raw, &parsed)) + + input, ok := parsed["input"].(map[string]any) + require.True(t, ok) + + kafka, ok := input["kafka_franz"].(map[string]any) + require.True(t, ok) + + _, hasTCP := kafka["tcp"] + assert.False(t, hasTCP, "tcp block should be absent when ConnectTimeout is 0") +} + func TestBenthosSASLObjectUsesPasswordFile(t *testing.T) { dir := t.TempDir() secretPath := filepath.Join(dir, "secret.txt") diff --git a/pkg/consumoor/source/config.go b/pkg/consumoor/source/config.go index f2f9ff85..e3d363e7 100644 --- a/pkg/consumoor/source/config.go +++ b/pkg/consumoor/source/config.go @@ -72,6 +72,11 @@ type KafkaConfig struct { // LagPollInterval controls how often consumer lag is polled from Kafka. // Set to 0 to disable lag monitoring. Default: 30s. LagPollInterval time.Duration `yaml:"lagPollInterval" default:"30s"` + // ConnectTimeout is the maximum time a TCP dial to a broker will wait + // for a connection to complete. A reasonable value (e.g. 10s) prevents + // hung dials from generating noisy warnings when some brokers are + // temporarily unreachable. Default: 10s. Set to 0 to disable. + ConnectTimeout time.Duration `yaml:"connectTimeout" default:"10s"` } // SASLConfig configures SASL authentication for Kafka. From 27fefae4c55d117528a9a7fbdadc385eb1664568 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 14:13:57 +1000 Subject: [PATCH 04/27] fix(consumoor): query lag for per-topic consumer groups instead of base group MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The lag monitor was querying Kafka for the base consumer group name (e.g. "xatu-consumoor") which doesn't exist — actual groups are per-topic suffixed (e.g. "xatu-consumoor-"). Pass the real group names so lag metrics are actually emitted. --- pkg/consumoor/consumoor.go | 7 +- pkg/consumoor/source/lag_monitor.go | 100 ++++++++++++++++------------ 2 files changed, 63 insertions(+), 44 deletions(-) diff --git a/pkg/consumoor/consumoor.go b/pkg/consumoor/consumoor.go index 472e0415..37225b73 100644 --- a/pkg/consumoor/consumoor.go +++ b/pkg/consumoor/consumoor.go @@ -151,7 +151,12 @@ func New( // Optionally create the Kafka consumer lag monitor. if config.Kafka.LagPollInterval > 0 { - lagMon, lagErr := source.NewLagMonitor(log, &config.Kafka, metrics) + consumerGroups := make([]string, 0, len(topics)) + for _, topic := range topics { + consumerGroups = append(consumerGroups, config.Kafka.ConsumerGroup+"-"+topic) + } + + lagMon, lagErr := source.NewLagMonitor(log, &config.Kafka, consumerGroups, metrics) if lagErr != nil { return nil, fmt.Errorf("creating lag monitor: %w", lagErr) } diff --git a/pkg/consumoor/source/lag_monitor.go b/pkg/consumoor/source/lag_monitor.go index b1701c91..5a2fe0d5 100644 --- a/pkg/consumoor/source/lag_monitor.go +++ b/pkg/consumoor/source/lag_monitor.go @@ -20,8 +20,8 @@ type LagMonitor struct { log logrus.FieldLogger metrics *telemetry.Metrics - interval time.Duration - consumerGroup string + interval time.Duration + consumerGroups []string admClient *kadm.Client kgoClient *kgo.Client @@ -30,15 +30,22 @@ type LagMonitor struct { } // NewLagMonitor creates a new LagMonitor. Call Start to begin polling. +// The consumerGroups slice should contain the actual per-topic consumer +// group names used by the Benthos streams (e.g. "base-group-topicA"). func NewLagMonitor( log logrus.FieldLogger, cfg *KafkaConfig, + consumerGroups []string, metrics *telemetry.Metrics, ) (*LagMonitor, error) { if cfg == nil { return nil, fmt.Errorf("nil kafka config") } + if len(consumerGroups) == 0 { + return nil, fmt.Errorf("no consumer groups to monitor") + } + opts := []kgo.Opt{ kgo.SeedBrokers(cfg.Brokers...), } @@ -67,14 +74,14 @@ func NewLagMonitor( } return &LagMonitor{ - log: log.WithField("component", "lag_monitor"), - metrics: metrics, - interval: cfg.LagPollInterval, - consumerGroup: cfg.ConsumerGroup, - admClient: kadm.NewClient(kgoClient), - kgoClient: kgoClient, - done: make(chan struct{}), - exited: make(chan struct{}), + log: log.WithField("component", "lag_monitor"), + metrics: metrics, + interval: cfg.LagPollInterval, + consumerGroups: consumerGroups, + admClient: kadm.NewClient(kgoClient), + kgoClient: kgoClient, + done: make(chan struct{}), + exited: make(chan struct{}), }, nil } @@ -111,51 +118,58 @@ func (m *LagMonitor) Stop() error { return nil } -// poll uses kadm.Client.Lag to fetch and publish consumer group lag. +// poll uses kadm.Client.Lag to fetch and publish consumer group lag +// for all per-topic consumer groups. func (m *LagMonitor) poll(ctx context.Context) { - lags, err := m.admClient.Lag(ctx, m.consumerGroup) + lags, err := m.admClient.Lag(ctx, m.consumerGroups...) if err != nil { m.log.WithError(err).Warn("Failed to fetch consumer group lag") return } - groupLag, ok := lags[m.consumerGroup] - if !ok { - m.log.Debug("Consumer group not found in lag response") + for _, group := range m.consumerGroups { + groupLag, ok := lags[group] + if !ok { + m.log.WithField("consumer_group", group). + Debug("Consumer group not found in lag response") - return - } - - if groupLag.Error() != nil { - m.log.WithError(groupLag.Error()).Warn("Error in consumer group lag response") - - return - } - - for topic, partitions := range groupLag.Lag { - for partition := range partitions { - ml := partitions[partition] + continue + } - if ml.Err != nil { - m.log.WithError(ml.Err). - WithField("topic", topic). - WithField("partition", partition). - Warn("Error computing lag for partition") + if groupLag.Error() != nil { + m.log.WithError(groupLag.Error()). + WithField("consumer_group", group). + Warn("Error in consumer group lag response") - continue - } + continue + } - lag := ml.Lag - if lag < 0 { - lag = 0 + for topic, partitions := range groupLag.Lag { + for partition := range partitions { + ml := partitions[partition] + + if ml.Err != nil { + m.log.WithError(ml.Err). + WithField("topic", topic). + WithField("partition", partition). + WithField("consumer_group", group). + Warn("Error computing lag for partition") + + continue + } + + lag := ml.Lag + if lag < 0 { + lag = 0 + } + + m.metrics.KafkaConsumerLag().WithLabelValues( + topic, + strconv.FormatInt(int64(partition), 10), + group, + ).Set(float64(lag)) } - - m.metrics.KafkaConsumerLag().WithLabelValues( - topic, - strconv.FormatInt(int64(partition), 10), - m.consumerGroup, - ).Set(float64(lag)) } } } From 13bdc194b90ea7fd9c19bf65f782200f73c71d9b Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 15:22:58 +1000 Subject: [PATCH 05/27] feat(consumoor): add Benthos output batching to fix 1-row INSERT bottleneck WriteBatch was called with 1 message at a time because service.BatchPolicy{} (zero value) applied no output batching. This made FlushTables write exactly 1 row per ClickHouse INSERT (~26ms each = ~38 rows/s), causing massive lag buildup. Add configurable outputBatchCount (default: 1000) and outputBatchPeriod (default: 1s) to KafkaConfig. Benthos now accumulates messages before calling WriteBatch, so each FlushTables writes ~1000 rows per INSERT instead of 1. --- pkg/consumoor/clickhouse/config.go | 4 +- pkg/consumoor/clickhouse/config_test.go | 4 +- pkg/consumoor/source/benthos.go | 12 ++- pkg/consumoor/source/benthos_test.go | 125 ++++++++++++++++++++++++ pkg/consumoor/source/config.go | 25 ++++- 5 files changed, 164 insertions(+), 6 deletions(-) diff --git a/pkg/consumoor/clickhouse/config.go b/pkg/consumoor/clickhouse/config.go index 4361799d..7fe8e7be 100644 --- a/pkg/consumoor/clickhouse/config.go +++ b/pkg/consumoor/clickhouse/config.go @@ -61,11 +61,11 @@ type Config struct { // TableConfig holds batching parameters for a ClickHouse table. type TableConfig struct { // BatchSize is the maximum number of rows per batch insert. - BatchSize int `yaml:"batchSize" default:"200000"` + BatchSize int `yaml:"batchSize" default:"50000"` // FlushInterval is the maximum time between flushes. FlushInterval time.Duration `yaml:"flushInterval" default:"1s"` // BufferSize is the channel buffer capacity for pending rows. - BufferSize int `yaml:"bufferSize" default:"200000"` + BufferSize int `yaml:"bufferSize" default:"50000"` // SkipFlattenErrors when true skips events that fail FlattenTo // instead of failing the entire batch. Default false = fail-fast. SkipFlattenErrors bool `yaml:"skipFlattenErrors"` diff --git a/pkg/consumoor/clickhouse/config_test.go b/pkg/consumoor/clickhouse/config_test.go index 55110637..2a392d41 100644 --- a/pkg/consumoor/clickhouse/config_test.go +++ b/pkg/consumoor/clickhouse/config_test.go @@ -15,9 +15,9 @@ func validConfig() Config { OrganicRetryMaxDelay: 30 * time.Second, DrainTimeout: 30 * time.Second, Defaults: TableConfig{ - BatchSize: 200000, + BatchSize: 50000, FlushInterval: 1 * time.Second, - BufferSize: 200000, + BufferSize: 50000, }, ChGo: validChGoConfig(), } diff --git a/pkg/consumoor/source/benthos.go b/pkg/consumoor/source/benthos.go index 27b87dd0..f18f7921 100644 --- a/pkg/consumoor/source/benthos.go +++ b/pkg/consumoor/source/benthos.go @@ -48,6 +48,15 @@ func NewBenthosStream( } } + batchPolicy := service.BatchPolicy{} + if kafkaConfig.OutputBatchCount > 0 { + batchPolicy.Count = kafkaConfig.OutputBatchCount + } + + if kafkaConfig.OutputBatchPeriod > 0 { + batchPolicy.Period = kafkaConfig.OutputBatchPeriod.String() + } + if registerErr := env.RegisterBatchOutput( benthosOutputType, service.NewConfigSpec(), @@ -60,7 +69,7 @@ func NewBenthosStream( metrics: metrics, rejectSink: rejectSink, ownsWriter: ownsWriter, - }, service.BatchPolicy{}, 1, nil + }, batchPolicy, 1, nil }, ); registerErr != nil { closeRejectSink() @@ -106,6 +115,7 @@ func benthosConfigYAML(logLevel string, kafkaConfig *KafkaConfig) ([]byte, error "fetch_min_bytes": fmt.Sprintf("%dB", kafkaConfig.FetchMinBytes), "fetch_max_wait": fmt.Sprintf("%dms", kafkaConfig.FetchWaitMaxMs), "fetch_max_partition_bytes": fmt.Sprintf("%dB", kafkaConfig.MaxPartitionFetchBytes), + "fetch_max_bytes": fmt.Sprintf("%dB", kafkaConfig.FetchMaxBytes), "session_timeout": fmt.Sprintf("%dms", kafkaConfig.SessionTimeoutMs), "heartbeat_interval": fmt.Sprintf("%dms", kafkaConfig.heartbeatIntervalMs()), } diff --git a/pkg/consumoor/source/benthos_test.go b/pkg/consumoor/source/benthos_test.go index b81aa2a2..fbafc196 100644 --- a/pkg/consumoor/source/benthos_test.go +++ b/pkg/consumoor/source/benthos_test.go @@ -145,6 +145,7 @@ func TestBenthosConfigYAML(t *testing.T) { FetchMinBytes: 64, FetchWaitMaxMs: 250, MaxPartitionFetchBytes: 1048576, + FetchMaxBytes: 5242880, SessionTimeoutMs: 30000, OffsetDefault: "latest", CommitInterval: 7 * time.Second, @@ -166,6 +167,8 @@ func TestBenthosConfigYAML(t *testing.T) { assert.Equal(t, "xatu-consumoor", kafka["consumer_group"]) assert.Equal(t, "latest", kafka["start_offset"]) assert.Equal(t, "7s", kafka["commit_period"]) + assert.Equal(t, "5242880B", kafka["fetch_max_bytes"]) + assert.Equal(t, "1048576B", kafka["fetch_max_partition_bytes"]) tcp, ok := kafka["tcp"].(map[string]any) require.True(t, ok, "tcp block should be present when ConnectTimeout > 0") @@ -187,6 +190,7 @@ func TestBenthosConfigYAML_NoTCPBlockWhenConnectTimeoutZero(t *testing.T) { FetchMinBytes: 1, FetchWaitMaxMs: 250, MaxPartitionFetchBytes: 1048576, + FetchMaxBytes: 10485760, SessionTimeoutMs: 30000, OffsetDefault: "earliest", CommitInterval: 5 * time.Second, @@ -562,6 +566,127 @@ func TestKafkaTopicMetadataFallback(t *testing.T) { assert.Equal(t, "xatu-mainnet", kafkaTopicMetadata(msg)) } +func TestKafkaConfig_Validate_OutputBatch(t *testing.T) { + validKafka := func() *KafkaConfig { + return &KafkaConfig{ + Brokers: []string{"kafka-1:9092"}, + Topics: []string{"^test-.+"}, + ConsumerGroup: "xatu-consumoor", + Encoding: "json", + FetchMinBytes: 1, + FetchWaitMaxMs: 250, + MaxPartitionFetchBytes: 1048576, + FetchMaxBytes: 10485760, + SessionTimeoutMs: 30000, + OffsetDefault: "earliest", + CommitInterval: 5 * time.Second, + ShutdownTimeout: 30 * time.Second, + OutputBatchCount: 1000, + OutputBatchPeriod: 1 * time.Second, + } + } + + tests := []struct { + name string + mutate func(*KafkaConfig) + wantErr string + }{ + { + name: "defaults valid", + mutate: func(_ *KafkaConfig) {}, + }, + { + name: "zero count valid (opt-out)", + mutate: func(c *KafkaConfig) { c.OutputBatchCount = 0 }, + }, + { + name: "zero period valid (opt-out)", + mutate: func(c *KafkaConfig) { c.OutputBatchPeriod = 0 }, + }, + { + name: "negative count", + mutate: func(c *KafkaConfig) { c.OutputBatchCount = -1 }, + wantErr: "outputBatchCount must be >= 0", + }, + { + name: "negative period", + mutate: func(c *KafkaConfig) { c.OutputBatchPeriod = -1 * time.Second }, + wantErr: "outputBatchPeriod must be >= 0", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + cfg := validKafka() + tt.mutate(cfg) + + err := cfg.Validate() + if tt.wantErr != "" { + require.Error(t, err) + assert.Contains(t, err.Error(), tt.wantErr) + } else { + require.NoError(t, err) + } + }) + } +} + +func TestBenthosOutputBatchPolicy(t *testing.T) { + tests := []struct { + name string + count int + period time.Duration + wantCount int + wantPeriod string + }{ + { + name: "defaults", + count: 1000, + period: 1 * time.Second, + wantCount: 1000, + wantPeriod: "1s", + }, + { + name: "zero count disables count batching", + count: 0, + period: 1 * time.Second, + wantCount: 0, + wantPeriod: "1s", + }, + { + name: "zero period disables period flushing", + count: 500, + period: 0, + wantCount: 500, + wantPeriod: "", + }, + { + name: "both zero preserves opt-out", + count: 0, + period: 0, + wantCount: 0, + wantPeriod: "", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + // Mirror the batch policy construction from NewBenthosStream. + policy := service.BatchPolicy{} + if tt.count > 0 { + policy.Count = tt.count + } + + if tt.period > 0 { + policy.Period = tt.period.String() + } + + assert.Equal(t, tt.wantCount, policy.Count) + assert.Equal(t, tt.wantPeriod, policy.Period) + }) + } +} + func TestFranzSASLMechanism(t *testing.T) { t.Run("defaults to PLAIN", func(t *testing.T) { mech, err := franzSASLMechanism(&SASLConfig{ diff --git a/pkg/consumoor/source/config.go b/pkg/consumoor/source/config.go index e3d363e7..6f63ef6e 100644 --- a/pkg/consumoor/source/config.go +++ b/pkg/consumoor/source/config.go @@ -47,7 +47,12 @@ type KafkaConfig struct { // FetchWaitMaxMs is the maximum time to wait for fetch responses. FetchWaitMaxMs int `yaml:"fetchWaitMaxMs" default:"250"` // MaxPartitionFetchBytes is the max bytes per partition per request. - MaxPartitionFetchBytes int32 `yaml:"maxPartitionFetchBytes" default:"10485760"` + MaxPartitionFetchBytes int32 `yaml:"maxPartitionFetchBytes" default:"3145728"` + // FetchMaxBytes is the max total bytes per fetch request across all + // partitions from a single broker. With many independent consumers + // (one per topic) this is the primary lever for capping in-flight + // memory from Kafka fetch buffers. Default: 10 MiB. + FetchMaxBytes int32 `yaml:"fetchMaxBytes" default:"10485760"` // SessionTimeoutMs is the consumer group session timeout. SessionTimeoutMs int `yaml:"sessionTimeoutMs" default:"30000"` @@ -77,6 +82,16 @@ type KafkaConfig struct { // hung dials from generating noisy warnings when some brokers are // temporarily unreachable. Default: 10s. Set to 0 to disable. ConnectTimeout time.Duration `yaml:"connectTimeout" default:"10s"` + + // OutputBatchCount is the number of messages Benthos accumulates before + // calling WriteBatch on the output plugin. Higher values increase INSERT + // throughput by writing more rows per ClickHouse INSERT. Set to 0 to + // disable count-based batching. Default: 1000. + OutputBatchCount int `yaml:"outputBatchCount" default:"1000"` + // OutputBatchPeriod is the maximum time Benthos waits to fill a batch + // before flushing a partial batch. Ensures low-volume topics still make + // progress. Default: 1s. Set to 0 to disable period-based flushing. + OutputBatchPeriod time.Duration `yaml:"outputBatchPeriod" default:"1s"` } // SASLConfig configures SASL authentication for Kafka. @@ -131,6 +146,14 @@ func (c *KafkaConfig) Validate() error { ) } + if c.OutputBatchCount < 0 { + return errors.New("kafka: outputBatchCount must be >= 0") + } + + if c.OutputBatchPeriod < 0 { + return errors.New("kafka: outputBatchPeriod must be >= 0") + } + if err := c.TLS.Validate(); err != nil { return fmt.Errorf("kafka.%w", err) } From 308d033026606b54055962b083a59249370b3890 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 17:14:01 +1000 Subject: [PATCH 06/27] feat(consumoor): remove per-table serialization, enable concurrent INSERTs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Remove the run() goroutine and flushReq channel that serialized per-table flushes. FlushTables now calls flushBuffer directly from the calling goroutine, enabling concurrent INSERTs bounded by the adaptive concurrency limiter (switched from TryAcquirePermit to blocking AcquirePermit). Each flush gets its own ColumnarBatch via newBatch() — no shared state. Drain is serialized via drainMu so the first goroutine gets a large batch while subsequent ones find an empty buffer and return nil. Defaults changed: adaptive limiter enabled by default, maxConns 32 -> 64. --- docs/consumoor-runbook.md | 65 ++-- example_consumoor.yaml | 26 +- pkg/consumoor/clickhouse/adaptive_limiter.go | 13 +- .../clickhouse/adaptive_limiter_test.go | 10 +- pkg/consumoor/clickhouse/config.go | 75 +---- pkg/consumoor/clickhouse/config_test.go | 85 +---- pkg/consumoor/clickhouse/table_writer.go | 290 +++--------------- pkg/consumoor/clickhouse/writer.go | 90 +++--- .../clickhouse/writer_benchmark_test.go | 77 +++-- pkg/consumoor/clickhouse/writer_test.go | 267 +++++----------- 10 files changed, 286 insertions(+), 712 deletions(-) diff --git a/docs/consumoor-runbook.md b/docs/consumoor-runbook.md index 46415776..95cd85f1 100644 --- a/docs/consumoor-runbook.md +++ b/docs/consumoor-runbook.md @@ -10,7 +10,7 @@ Consumoor is a Kafka-to-ClickHouse pipeline that reads Ethereum network events f Kafka topics | v -Benthos (kafka_franz input) +Benthos (kafka_franz input, maxInFlight=64 concurrent WriteBatch calls) | v Decoder (JSON or Protobuf -> DecoratedEvent) @@ -22,17 +22,20 @@ Router Engine (event name -> target table(s)) Per-table buffer (bounded channel, one per table) | v -Table Writer (organic + coordinated flush) +FlushTables (concurrent drain + flush from calling goroutines) | v -ClickHouse (ch-go columnar inserts) +Adaptive Concurrency Limiter (per-table AIMD, blocking acquire) + | + v +ClickHouse (ch-go columnar inserts, concurrent per table) ``` **Key design decisions:** - **At-least-once delivery.** Kafka offsets are committed periodically (`commitInterval`, default 5s). On crash, messages since the last commit are replayed. ClickHouse must tolerate duplicate inserts (e.g. via `ReplacingMergeTree`). - **Per-topic consumer groups.** Each consumoor instance joins a single consumer group. Multiple instances with different consumer groups or topic patterns can run in parallel. -- **Organic vs coordinated flushes.** Each table writer flushes independently when its batch reaches `batchSize` rows or when `flushInterval` elapses (organic). The Benthos output plugin can also trigger a coordinated `FlushAll` after writing a batch of messages, depending on the `deliveryMode`. +- **Benthos-driven batching.** Benthos accumulates messages according to `outputBatchCount` and `outputBatchPeriod`, then calls `WriteBatch` which routes events to per-table buffers and triggers a synchronous flush. Up to `maxInFlight` (default 64) WriteBatch calls run concurrently, enabling concurrent INSERTs per table bounded by the adaptive concurrency limiter. There are no independent timer-based flushes -- all writes are coordinated through Benthos. - **Backpressure propagation.** Each table's buffer is a bounded channel (`bufferSize`). When a buffer fills (e.g. ClickHouse is slow), `Write()` blocks, which backs up the Benthos output, which backs up Kafka consumption. This prevents unbounded memory growth. - **Error classification.** Write errors are classified as permanent (schema mismatch, unknown table, type errors) or transient (network, timeout, overload). Permanent errors drop the batch and optionally send messages to a DLQ topic. Transient errors trigger retry with exponential backoff. @@ -58,8 +61,7 @@ All metrics use the prefix `xatu_consumoor_`. | `rows_written_total` | Counter | `table` | Rows successfully written to ClickHouse | Should increase; flat = writes stalled | | `write_errors_total` | Counter | `table` | ClickHouse write errors | Any increase; check permanent vs transient | | `write_duration_seconds` | Histogram | `table` | Duration of batch inserts | p99 > `queryTimeout` = ClickHouse too slow | -| `batch_size` | Histogram | `table` | Rows per batch write | Very small batches = inefficient; tune `batchSize` | -| `buffer_usage` | Gauge | `table` | Current buffered rows per table | Approaching `bufferSize` = backpressure building | +| `batch_size` | Histogram | `table` | Rows per batch write | Very small batches = inefficient; tune `outputBatchCount` | ### DLQ Metrics @@ -88,7 +90,6 @@ All metrics use the prefix `xatu_consumoor_`. ### ClickHouse Down or Slow **Symptoms:** -- `buffer_usage` climbing toward `bufferSize` for affected tables - `write_errors_total` increasing - `rows_written_total` flat - `write_duration_seconds` p99 increasing (if ClickHouse is slow but reachable) @@ -96,16 +97,14 @@ All metrics use the prefix `xatu_consumoor_`. **What happens internally:** 1. Table writer flush fails with a transient error (connection refused, timeout, etc.) -2. The table writer enters `flushBlocked` state -- it stops draining its buffer channel and preserves the failed batch for retry -3. Buffer channel fills up, blocking `Write()` calls -4. Benthos output blocks, causing Kafka fetch to stall -5. On next `flushInterval` tick, the table writer retries the pending batch (with exponential backoff via `doWithRetry`: default 3 retries, 100ms base delay, 2s max delay) -6. If retry succeeds, `flushBlocked` clears and normal processing resumes +2. `FlushTables` returns the error to `processGroup`, which NAKs the Kafka messages (they will be re-delivered by Kafka) +3. Each flush attempt uses exponential backoff via `doWithRetry` (default 3 retries, 100ms base delay, 2s max delay) +4. The buffer channel fills up, blocking `Write()` calls, which backs up Benthos output and stalls Kafka consumption **Recovery:** - Fix ClickHouse (restart, add capacity, resolve disk issues) - Consumoor auto-recovers once ClickHouse is available -- no restart needed -- Monitor `buffer_usage` dropping and `rows_written_total` resuming +- Monitor `rows_written_total` resuming - Expect a burst of writes as buffered data flushes ### Kafka Rebalance Storm @@ -130,27 +129,24 @@ All metrics use the prefix `xatu_consumoor_`. ### OOM / Memory Pressure **Symptoms:** -- `buffer_usage` high across multiple tables - Pod OOMKilled restarts - Memory usage on the node spiking **Causes:** - ClickHouse backpressure fills all table buffers simultaneously. Total potential memory = `bufferSize * number_of_active_tables * avg_event_size` - Very large `bufferSize` values combined with many tables -- Large batch sizes holding many events in memory during flush +- Large `outputBatchCount` holding many events in memory during flush **Mitigation:** - Reduce `bufferSize` in defaults (e.g. from 200000 to 50000) +- Reduce `outputBatchCount` to flush smaller batches - Increase ClickHouse write capacity (more replicas, faster disks) -- Tune `batchSize` down to flush more frequently with less memory per flush - Increase pod memory limits if the workload genuinely requires it -- Monitor `buffer_usage` across all tables to estimate actual memory use ### Flatten Livelock **Symptoms:** - One table's `rows_written_total` stops increasing -- `buffer_usage` for that table grows (or stays at max if already full) - `write_errors_total` for that table climbing - `flatten_errors_total` increasing for the affected event/table - Other tables continue writing normally @@ -175,16 +171,15 @@ All metrics use the prefix `xatu_consumoor_`. **Causes:** - ClickHouse write throughput is the bottleneck (check `write_duration_seconds`) -- Batch sizes too small causing excessive round-trips (check `batch_size` histogram) +- Batch sizes too small causing excessive round-trips (check `batch_size` histogram; increase `outputBatchCount`) - Single instance hitting connection pool limits (check `chgo_pool_empty_acquire_total`) - High-cardinality topics spreading events across many tables, each with its own flush cycle **Tuning:** -- Increase `batchSize` to write more rows per round-trip (e.g. 200000 -> 500000) +- Increase `outputBatchCount` to write more rows per round-trip (e.g. 1000 -> 5000) - Increase `maxConns` to allow more concurrent ClickHouse writes - Increase ClickHouse resources (CPU, memory, disk I/O) - Scale horizontally: run multiple consumoor instances with different topic patterns or consumer groups -- Switch from `deliveryMode: message` to `deliveryMode: batch` if not already (batch mode is significantly faster) ### DLQ Failures @@ -265,9 +260,9 @@ Common causes of startup failure: ### Vertical Scaling -- Increase `maxConns` to allow more concurrent ClickHouse writes (default: 32) -- Increase `batchSize` to write more rows per INSERT (default: 200000) -- Increase `bufferSize` to absorb more backpressure spikes (default: 200000), at the cost of higher memory usage +- Increase `maxConns` to allow more concurrent ClickHouse writes (default: 64) +- Increase `outputBatchCount` to write more rows per INSERT (default: 1000) +- Increase `bufferSize` to absorb more backpressure spikes (default: 50000), at the cost of higher memory usage - Increase pod CPU/memory to handle more concurrent table writers ## Configuration Tuning @@ -276,34 +271,32 @@ Common causes of startup failure: | Parameter | Path | Default | When to adjust | |-----------|------|---------|----------------| -| `batchSize` | `clickhouse.defaults.batchSize` | 200000 | Increase for higher throughput (more rows per INSERT); decrease to reduce memory per flush | -| `flushInterval` | `clickhouse.defaults.flushInterval` | 1s | Decrease for lower latency; increase if batches are too small | -| `bufferSize` | `clickhouse.defaults.bufferSize` | 200000 | Increase to absorb ClickHouse hiccups without backpressure; decrease to limit memory | +| `outputBatchCount` | `kafka.outputBatchCount` | 1000 | Increase for higher throughput (more rows per INSERT); decrease to reduce memory per flush | +| `outputBatchPeriod` | `kafka.outputBatchPeriod` | 1s | Decrease for lower latency; increase if batches are too small | +| `maxInFlight` | `kafka.maxInFlight` | 64 | Concurrent WriteBatch calls per stream; increase for more throughput, decrease if ClickHouse is overloaded | +| `bufferSize` | `clickhouse.defaults.bufferSize` | 50000 | Increase to absorb ClickHouse hiccups without backpressure; decrease to limit memory | | `commitInterval` | `kafka.commitInterval` | 5s | Decrease to reduce duplicate replay window on crash; increase to reduce Kafka commit overhead | -| `maxConns` | `clickhouse.chgo.maxConns` | 32 | Increase when `chgo_pool_empty_acquire_total` is high | +| `maxConns` | `clickhouse.chgo.maxConns` | 64 | Increase when `chgo_pool_empty_acquire_total` is high | | `queryTimeout` | `clickhouse.chgo.queryTimeout` | 30s | Increase if large batches legitimately take longer to insert | | `maxRetries` | `clickhouse.chgo.maxRetries` | 3 | Increase if transient ClickHouse errors are frequent but recover quickly | -| `deliveryMode` | `kafka.deliveryMode` | batch | Use `message` for safer per-message delivery; use `batch` for higher throughput | | `sessionTimeoutMs` | `kafka.sessionTimeoutMs` | 30000 | Increase if rebalances are frequent due to slow processing | +| `adaptiveLimiter.enabled` | `clickhouse.chgo.adaptiveLimiter.enabled` | true | Per-table AIMD concurrency limiting; disable if all tables have uniform latency | +| `adaptiveLimiter.initialLimit` | `clickhouse.chgo.adaptiveLimiter.initialLimit` | 8 | Starting concurrent INSERTs per table before adaptation | +| `adaptiveLimiter.maxLimit` | `clickhouse.chgo.adaptiveLimiter.maxLimit` | 50 | Upper bound on concurrent INSERTs per table | | `tableSuffix` | `clickhouse.tableSuffix` | (empty) | Set to `_local` to bypass Distributed tables in clustered setups | ### Per-Table Overrides -High-volume tables (e.g. attestations, committees) may need larger batch and buffer sizes: +High-volume tables (e.g. attestations, committees) may need larger buffer sizes: ```yaml clickhouse: defaults: - batchSize: 200000 - bufferSize: 200000 - flushInterval: 1s + bufferSize: 50000 tables: beacon_api_eth_v1_events_attestation: - batchSize: 1000000 bufferSize: 1000000 - flushInterval: 5s beacon_api_eth_v1_beacon_committee: - batchSize: 1000000 bufferSize: 1000000 ``` diff --git a/example_consumoor.yaml b/example_consumoor.yaml index 27386be7..25ab787b 100644 --- a/example_consumoor.yaml +++ b/example_consumoor.yaml @@ -12,14 +12,17 @@ kafka: # offsetDefault: earliest # fetchMinBytes: 1 # fetchWaitMaxMs: 500 - # maxPartitionFetchBytes: 10485760 + # maxPartitionFetchBytes: 3145728 + # fetchMaxBytes: 10485760 # sessionTimeoutMs: 30000 - # heartbeatIntervalMs: 3000 commitInterval: 5s # Kafka offset commit interval # shutdownTimeout: 30s # max wait for in-flight messages during graceful shutdown # lagPollInterval: 30s # consumer lag poll interval (0 to disable) # rejectedTopic: xatu-consumoor-rejected # topicRefreshInterval: 60s # How often to re-discover topics matching regex patterns (0 to disable) + # outputBatchCount: 1000 # messages per Benthos output batch (0 to disable) + # outputBatchPeriod: 1s # max wait before flushing partial batch + # maxInFlight: 64 # concurrent WriteBatch calls per stream (higher = more throughput) # tls: # enabled: false # caFile: /path/to/ca.pem @@ -33,8 +36,6 @@ kafka: clickhouse: dsn: "clickhouse://localhost:9000/default" - # organicRetryInitDelay: 1s # initial backoff when a table flush fails - # organicRetryMaxDelay: 30s # max backoff cap for organic retries # failOnMissingTables: true # set to false to downgrade missing table errors to warnings # tls: # enabled: false @@ -49,29 +50,30 @@ clickhouse: # maxRetries: 3 # retryBaseDelay: 100ms # retryMaxDelay: 2s - # maxConns: 32 + # maxConns: 64 # minConns: 1 # connMaxLifetime: 1h # connMaxIdleTime: 10m # healthCheckPeriod: 30s # poolMetricsInterval: 15s - # bufferWarningThreshold: 0.8 # log warning when any table buffer exceeds this fraction (0-1) + # adaptiveLimiter: + # enabled: true # per-table adaptive concurrency limiting (AIMD) + # minLimit: 1 # minimum concurrent INSERTs per table + # maxLimit: 50 # maximum concurrent INSERTs per table + # initialLimit: 8 # starting concurrency before adaptation + # queueInitialRejectionFactor: 2 + # queueMaxRejectionFactor: 3 defaults: - batchSize: 200000 - flushInterval: 1s - bufferSize: 200000 + bufferSize: 50000 tables: # Canonical tables default to insertSettings.insert_quorum: auto (majority). # Set insertSettings.insert_quorum explicitly to override that behavior. beacon_api_eth_v1_beacon_committee: - batchSize: 1000000 bufferSize: 1000000 # insertSettings: # insert_quorum: 2 # insert_quorum_timeout: 60000 beacon_api_eth_v1_events_attestation: - batchSize: 1000000 - flushInterval: 5s bufferSize: 1000000 # canonical_beacon_block: # insertSettings: diff --git a/pkg/consumoor/clickhouse/adaptive_limiter.go b/pkg/consumoor/clickhouse/adaptive_limiter.go index 176e3b49..24b4aec7 100644 --- a/pkg/consumoor/clickhouse/adaptive_limiter.go +++ b/pkg/consumoor/clickhouse/adaptive_limiter.go @@ -51,7 +51,8 @@ func (l *adaptiveConcurrencyLimiter) Queued() int { return l.limiter.Queued() } -// doWithLimiter acquires a permit, executes fn, and records the outcome. +// doWithLimiter acquires a permit (blocking until one is available or ctx +// is cancelled), executes fn, and records the outcome. // On success the permit is recorded (RTT measured); on failure the permit // is dropped (signals the limiter to decrease concurrency). // Returns limiterRejectedError if the limiter rejects the request. @@ -59,14 +60,12 @@ func (l *adaptiveConcurrencyLimiter) doWithLimiter( ctx context.Context, fn func(context.Context) error, ) error { - permit, ok := l.limiter.TryAcquirePermit() - if !ok { - return &limiterRejectedError{ - cause: adaptivelimiter.ErrExceeded, - } + permit, err := l.limiter.AcquirePermit(ctx) + if err != nil { + return &limiterRejectedError{cause: err} } - err := fn(ctx) + err = fn(ctx) if err != nil { permit.Drop() diff --git a/pkg/consumoor/clickhouse/adaptive_limiter_test.go b/pkg/consumoor/clickhouse/adaptive_limiter_test.go index ebe48f39..3d935316 100644 --- a/pkg/consumoor/clickhouse/adaptive_limiter_test.go +++ b/pkg/consumoor/clickhouse/adaptive_limiter_test.go @@ -4,6 +4,7 @@ import ( "context" "errors" "testing" + "time" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -74,7 +75,7 @@ func TestAdaptiveConcurrencyLimiter_AcquireAndDropOnError(t *testing.T) { assert.Equal(t, 0, limiter.Inflight()) } -func TestAdaptiveConcurrencyLimiter_RejectsWhenFull(t *testing.T) { +func TestAdaptiveConcurrencyLimiter_BlocksWhenFullAndCancels(t *testing.T) { // Create a limiter with limit=1 so we can saturate it. limiter := newAdaptiveConcurrencyLimiter(AdaptiveLimiterConfig{ Enabled: true, @@ -89,8 +90,11 @@ func TestAdaptiveConcurrencyLimiter_RejectsWhenFull(t *testing.T) { permit, ok := limiter.limiter.TryAcquirePermit() require.True(t, ok, "should acquire the single permit") - // Second attempt should be rejected. - err := limiter.doWithLimiter(context.Background(), func(_ context.Context) error { + // Second attempt should block and then fail when context is cancelled. + ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) + defer cancel() + + err := limiter.doWithLimiter(ctx, func(_ context.Context) error { return nil }) diff --git a/pkg/consumoor/clickhouse/config.go b/pkg/consumoor/clickhouse/config.go index 7fe8e7be..cbeebd5e 100644 --- a/pkg/consumoor/clickhouse/config.go +++ b/pkg/consumoor/clickhouse/config.go @@ -25,12 +25,6 @@ type Config struct { // directly to ReplicatedMergeTree tables in a clustered setup. TableSuffix string `yaml:"tableSuffix"` - // OrganicRetryInitDelay is the initial backoff delay when a table writer - // flush fails and the batch is preserved for retry on the next cycle. - OrganicRetryInitDelay time.Duration `yaml:"organicRetryInitDelay" default:"1s"` - // OrganicRetryMaxDelay caps the exponential backoff for organic retries. - OrganicRetryMaxDelay time.Duration `yaml:"organicRetryMaxDelay" default:"30s"` - // FailOnMissingTables controls whether missing ClickHouse tables cause // a fatal startup error. When true (default), startup is aborted if any // registered route table does not exist in the target database. Set to @@ -42,13 +36,7 @@ type Config struct { // cancelled after this duration rather than hanging indefinitely. DrainTimeout time.Duration `yaml:"drainTimeout" default:"30s"` - // BufferWarningThreshold is the fraction (0-1) of a table's bufferSize - // at which a rate-limited warning is logged. Provides early visibility - // into memory pressure before full backpressure kicks in. - // Default: 0.8 - BufferWarningThreshold float64 `yaml:"bufferWarningThreshold" default:"0.8"` - - // Defaults are the default batch settings for all tables. + // Defaults are the default table settings. Defaults TableConfig `yaml:"defaults"` // Tables contains per-table overrides for batch settings. @@ -58,13 +46,11 @@ type Config struct { ChGo ChGoConfig `yaml:"chgo"` } -// TableConfig holds batching parameters for a ClickHouse table. +// TableConfig holds per-table settings for the ClickHouse writer. type TableConfig struct { - // BatchSize is the maximum number of rows per batch insert. - BatchSize int `yaml:"batchSize" default:"50000"` - // FlushInterval is the maximum time between flushes. - FlushInterval time.Duration `yaml:"flushInterval" default:"1s"` // BufferSize is the channel buffer capacity for pending rows. + // When the buffer is full, Write() blocks, propagating backpressure + // through Benthos to the Kafka consumer. BufferSize int `yaml:"bufferSize" default:"50000"` // SkipFlattenErrors when true skips events that fail FlattenTo // instead of failing the entire batch. Default false = fail-fast. @@ -83,9 +69,8 @@ type TableConfig struct { // When enabled, each table writer independently adjusts its INSERT concurrency // based on observed ClickHouse latency using an AIMD algorithm. type AdaptiveLimiterConfig struct { - // Enabled turns on adaptive concurrency limiting. Disabled by default - // for zero behavior change unless explicitly opted in. - Enabled bool `yaml:"enabled" default:"false"` + // Enabled turns on adaptive concurrency limiting. + Enabled bool `yaml:"enabled" default:"true"` // MinLimit is the minimum concurrent INSERTs the limiter allows. MinLimit uint `yaml:"minLimit" default:"1"` // MaxLimit caps the maximum concurrent INSERTs the limiter allows. @@ -160,7 +145,7 @@ type ChGoConfig struct { RetryMaxDelay time.Duration `yaml:"retryMaxDelay" default:"2s"` // MaxConns is the maximum number of pooled ClickHouse connections. - MaxConns int32 `yaml:"maxConns" default:"32"` + MaxConns int32 `yaml:"maxConns" default:"64"` // MinConns is the minimum number of pooled ClickHouse connections. MinConns int32 `yaml:"minConns" default:"1"` // ConnMaxLifetime is the maximum lifetime for pooled connections. @@ -188,20 +173,6 @@ func (c *Config) Validate() error { return fmt.Errorf("clickhouse.%w", err) } - if c.OrganicRetryInitDelay <= 0 { - return errors.New("clickhouse: organicRetryInitDelay must be > 0") - } - - if c.OrganicRetryMaxDelay <= 0 { - return errors.New("clickhouse: organicRetryMaxDelay must be > 0") - } - - if c.OrganicRetryInitDelay > c.OrganicRetryMaxDelay { - return errors.New( - "clickhouse: organicRetryInitDelay must be <= organicRetryMaxDelay", - ) - } - if err := c.ChGo.Validate(); err != nil { return err } @@ -210,39 +181,15 @@ func (c *Config) Validate() error { return errors.New("clickhouse: drainTimeout must be > 0") } - if c.BufferWarningThreshold < 0 || c.BufferWarningThreshold > 1 { - return errors.New("clickhouse: bufferWarningThreshold must be between 0 and 1") - } - - if c.Defaults.BatchSize <= 0 { - return errors.New("clickhouse.defaults: batchSize must be > 0") - } - - if c.Defaults.FlushInterval <= 0 { - return errors.New("clickhouse.defaults: flushInterval must be > 0") - } - if c.Defaults.BufferSize <= 0 { return errors.New("clickhouse.defaults: bufferSize must be > 0") } - if c.Defaults.BufferSize < c.Defaults.BatchSize { - return errors.New("clickhouse.defaults: bufferSize must be >= batchSize") - } - if err := validateInsertSettings(c.Defaults.InsertSettings, "clickhouse.defaults.insertSettings"); err != nil { return err } for table, override := range c.Tables { - if override.BatchSize < 0 { - return fmt.Errorf("clickhouse.tables.%s: batchSize must be >= 0", table) - } - - if override.FlushInterval < 0 { - return fmt.Errorf("clickhouse.tables.%s: flushInterval must be >= 0", table) - } - if override.BufferSize < 0 { return fmt.Errorf("clickhouse.tables.%s: bufferSize must be >= 0", table) } @@ -330,14 +277,6 @@ func (c *Config) TableConfigFor(table string) TableConfig { return cfg } - if override.BatchSize > 0 { - cfg.BatchSize = override.BatchSize - } - - if override.FlushInterval > 0 { - cfg.FlushInterval = override.FlushInterval - } - if override.BufferSize > 0 { cfg.BufferSize = override.BufferSize } diff --git a/pkg/consumoor/clickhouse/config_test.go b/pkg/consumoor/clickhouse/config_test.go index 2a392d41..f6ead62b 100644 --- a/pkg/consumoor/clickhouse/config_test.go +++ b/pkg/consumoor/clickhouse/config_test.go @@ -8,21 +8,6 @@ import ( "github.com/stretchr/testify/require" ) -func validConfig() Config { - return Config{ - DSN: "clickhouse://localhost:9000/default", - OrganicRetryInitDelay: 1 * time.Second, - OrganicRetryMaxDelay: 30 * time.Second, - DrainTimeout: 30 * time.Second, - Defaults: TableConfig{ - BatchSize: 50000, - FlushInterval: 1 * time.Second, - BufferSize: 50000, - }, - ChGo: validChGoConfig(), - } -} - func validChGoConfig() ChGoConfig { return ChGoConfig{ DialTimeout: 5 * time.Second, @@ -31,12 +16,20 @@ func validChGoConfig() ChGoConfig { MaxRetries: 3, RetryBaseDelay: 100 * time.Millisecond, RetryMaxDelay: 2 * time.Second, - MaxConns: 32, + MaxConns: 64, MinConns: 1, ConnMaxLifetime: 1 * time.Hour, ConnMaxIdleTime: 10 * time.Minute, HealthCheckPeriod: 30 * time.Second, PoolMetricsInterval: 15 * time.Second, + AdaptiveLimiter: AdaptiveLimiterConfig{ + Enabled: true, + MinLimit: 1, + MaxLimit: 50, + InitialLimit: 8, + QueueInitialRejectionFactor: 2, + QueueMaxRejectionFactor: 3, + }, } } @@ -116,66 +109,6 @@ func TestChGoConfig_Validate_ReadTimeout(t *testing.T) { } } -func TestConfig_Validate_OrganicRetry(t *testing.T) { - tests := []struct { - name string - initDelay time.Duration - maxDelay time.Duration - wantErr string - }{ - { - name: "valid", - initDelay: 1 * time.Second, - maxDelay: 30 * time.Second, - }, - { - name: "equal init and max", - initDelay: 5 * time.Second, - maxDelay: 5 * time.Second, - }, - { - name: "zero init delay", - initDelay: 0, - maxDelay: 30 * time.Second, - wantErr: "organicRetryInitDelay must be > 0", - }, - { - name: "negative init delay", - initDelay: -1 * time.Second, - maxDelay: 30 * time.Second, - wantErr: "organicRetryInitDelay must be > 0", - }, - { - name: "zero max delay", - initDelay: 1 * time.Second, - maxDelay: 0, - wantErr: "organicRetryMaxDelay must be > 0", - }, - { - name: "init exceeds max", - initDelay: 30 * time.Second, - maxDelay: 1 * time.Second, - wantErr: "organicRetryInitDelay must be <= organicRetryMaxDelay", - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - cfg := validConfig() - cfg.OrganicRetryInitDelay = tt.initDelay - cfg.OrganicRetryMaxDelay = tt.maxDelay - - err := cfg.Validate() - if tt.wantErr != "" { - require.Error(t, err) - assert.Contains(t, err.Error(), tt.wantErr) - } else { - require.NoError(t, err) - } - }) - } -} - func TestAdaptiveLimiterConfig_Validate(t *testing.T) { tests := []struct { name string diff --git a/pkg/consumoor/clickhouse/table_writer.go b/pkg/consumoor/clickhouse/table_writer.go index 9bad6c6f..34aa8620 100644 --- a/pkg/consumoor/clickhouse/table_writer.go +++ b/pkg/consumoor/clickhouse/table_writer.go @@ -4,7 +4,7 @@ import ( "context" "errors" "fmt" - "sync/atomic" + "sync" "time" "github.com/ClickHouse/ch-go" @@ -14,9 +14,6 @@ import ( "github.com/sirupsen/logrus" ) -// bufferWarningInterval is the minimum time between warning logs per table. -const bufferWarningInterval = time.Minute - // eventEntry holds a single event for buffering in the table writer. type eventEntry struct { event *xatu.DecoratedEvent @@ -31,202 +28,76 @@ type chTableWriter struct { metrics *telemetry.Metrics writer *ChGoWriter buffer chan eventEntry - flushReq chan chan error // coordinator sends a response channel; tableWriter flushes and replies - organicRetryInitDelay time.Duration - organicRetryMaxDelay time.Duration - drainTimeout time.Duration + drainTimeout time.Duration + + // drainMu serializes buffer drains so that the first goroutine gets a + // large batch while subsequent concurrent flushers find an empty buffer. + drainMu sync.Mutex newBatch func() route.ColumnarBatch - batch route.ColumnarBatch // reused across flushes via Reset() // limiter is the per-table adaptive concurrency limiter. // nil when adaptive limiting is disabled. limiter *adaptiveConcurrencyLimiter - - // lastWarnAt tracks the last time a buffer warning was emitted (Unix nanos). - // Accessed atomically from the Write goroutine for rate limiting. - lastWarnAt atomic.Int64 } -func (tw *chTableWriter) run(done <-chan struct{}) { - ticker := time.NewTicker(tw.config.FlushInterval) - defer ticker.Stop() +// flushBuffer drains the buffer channel and flushes all accumulated events. +// The drain is serialized via drainMu so that the first caller gets a large +// batch while concurrent callers find an empty buffer and return nil. +func (tw *chTableWriter) flushBuffer(ctx context.Context) error { + tw.drainMu.Lock() - events := make([]eventEntry, 0, tw.config.BatchSize) - flushBlocked := false - retryAttempt := 0 + events := make([]eventEntry, 0, len(tw.buffer)) for { - bufferCh := tw.buffer - if flushBlocked { - bufferCh = nil - } - select { - case entry := <-bufferCh: - tw.decrBuffer(1) - + case entry := <-tw.buffer: events = append(events, entry) + default: + goto drained + } + } - if len(events) >= tw.config.BatchSize { - if err := tw.flush(context.Background(), events); err != nil { - var flatErr *flattenError - if errors.As(err, &flatErr) { - events = events[:0] - - continue - } - - if IsPermanentWriteError(err) { - tw.log.WithError(err). - WithField("events", len(events)). - Warn("Dropping permanently invalid batch") - events = events[:0] - flushBlocked = false - retryAttempt = 0 - - continue - } - - if IsLimiterRejected(err) { - flushBlocked = true - - continue - } - - // Preserve events for retry on next flush cycle. While - // pending, stop draining tw.buffer so channel backpressure - // remains bounded by BufferSize. - flushBlocked = true - - tw.scheduleOrganicRetry(ticker, retryAttempt) - retryAttempt++ - - continue - } - - events = events[:0] - flushBlocked = false - retryAttempt = 0 - } - - case <-ticker.C: - if len(events) > 0 { - if err := tw.flush(context.Background(), events); err != nil { - var flatErr *flattenError - if errors.As(err, &flatErr) { - events = events[:0] - - continue - } - - if IsPermanentWriteError(err) { - tw.log.WithError(err). - WithField("events", len(events)). - Warn("Dropping permanently invalid batch") - events = events[:0] - flushBlocked = false - - tw.resetRetryState(ticker, &retryAttempt) - - continue - } - - if IsLimiterRejected(err) { - flushBlocked = true - - continue - } - - flushBlocked = true +drained: - tw.scheduleOrganicRetry(ticker, retryAttempt) - retryAttempt++ + tw.drainMu.Unlock() - continue - } + if len(events) == 0 { + return nil + } - events = events[:0] - flushBlocked = false + return tw.flush(ctx, events) +} - tw.resetRetryState(ticker, &retryAttempt) - } +// drainAndFlush drains the buffer and flushes with a timeout context. +// Used during shutdown to flush any remaining events. +func (tw *chTableWriter) drainAndFlush(timeout time.Duration) error { + tw.drainMu.Lock() - case errCh := <-tw.flushReq: - if !flushBlocked { - // Drain anything still in the channel into the batch. - drainReq: - for { - select { - case entry := <-tw.buffer: - tw.decrBuffer(1) - - events = append(events, entry) - default: - break drainReq - } - } - } + events := make([]eventEntry, 0, len(tw.buffer)) - if len(events) > 0 { - err := tw.flush(context.Background(), events) - - var flatErr *flattenError - - switch { - case err == nil: - events = events[:0] - flushBlocked = false - - tw.resetRetryState(ticker, &retryAttempt) - case errors.As(err, &flatErr): - events = events[:0] - case IsPermanentWriteError(err): - tw.log.WithError(err). - WithField("events", len(events)). - Warn("Dropping permanently invalid batch") - events = events[:0] - flushBlocked = false - - tw.resetRetryState(ticker, &retryAttempt) - case IsLimiterRejected(err): - flushBlocked = true - default: - flushBlocked = true - } - // On error: events are PRESERVED for retry on next cycle - errCh <- err - } else { - errCh <- nil - } + for { + select { + case entry := <-tw.buffer: + events = append(events, entry) + default: + goto drained + } + } - case <-done: - drainLoop: - for { - select { - case entry := <-tw.buffer: - tw.decrBuffer(1) - - events = append(events, entry) - default: - break drainLoop - } - } +drained: - if len(events) > 0 { - drainCtx, drainCancel := context.WithTimeout(context.Background(), tw.drainTimeout) + tw.drainMu.Unlock() - if err := tw.flush(drainCtx, events); err != nil { - tw.log.WithError(err).WithField("events", len(events)).Warn("Flush error during shutdown drain") - } + if len(events) == 0 { + return nil + } - drainCancel() - } + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() - return - } - } + return tw.flush(ctx, events) } func (tw *chTableWriter) flush(ctx context.Context, events []eventEntry) error { @@ -247,13 +118,7 @@ func (tw *chTableWriter) flush(ctx context.Context, events []eventEntry) error { } } - if tw.batch == nil { - tw.batch = tw.newBatch() - } else { - tw.batch.Reset() - } - - batch := tw.batch + batch := tw.newBatch() var ( flattenErrs int @@ -355,31 +220,6 @@ func (tw *chTableWriter) flush(ctx context.Context, events []eventEntry) error { return nil } -// scheduleOrganicRetry resets the ticker interval to an exponential backoff -// delay so the next ticker.C fires after the computed retry delay. -func (tw *chTableWriter) scheduleOrganicRetry(ticker *time.Ticker, attempt int) { - delay := min( - tw.organicRetryInitDelay*time.Duration(1< 0 { - ticker.Reset(tw.config.FlushInterval) - } - - *attempt = 0 -} - func (tw *chTableWriter) do( ctx context.Context, operation string, @@ -411,41 +251,3 @@ func (tw *chTableWriter) do( return tw.limiter.doWithLimiter(attemptCtx, poolFn) }) } - -// checkBufferWarning emits a rate-limited warning when the buffer usage -// exceeds the configured BufferWarningThreshold. Safe to call from any -// goroutine. -func (tw *chTableWriter) checkBufferWarning() { - threshold := tw.writer.config.BufferWarningThreshold - if threshold <= 0 { - return - } - - current := len(tw.buffer) - limit := tw.config.BufferSize - - if float64(current) < threshold*float64(limit) { - return - } - - now := time.Now().UnixNano() - last := tw.lastWarnAt.Load() - - if now-last < int64(bufferWarningInterval) { - return - } - - if tw.lastWarnAt.CompareAndSwap(last, now) { - tw.log.WithFields(logrus.Fields{ - "current": current, - "max": limit, - "threshold": threshold, - }).Warn("Buffer usage exceeds warning threshold") - } -} - -// decrBuffer decrements both the per-table and aggregate buffer gauges. -func (tw *chTableWriter) decrBuffer(n int) { - tw.metrics.BufferUsage().WithLabelValues(tw.table).Sub(float64(n)) - tw.metrics.BufferUsageTotal().Sub(float64(n)) -} diff --git a/pkg/consumoor/clickhouse/writer.go b/pkg/consumoor/clickhouse/writer.go index 48cf8522..9fe1b746 100644 --- a/pkg/consumoor/clickhouse/writer.go +++ b/pkg/consumoor/clickhouse/writer.go @@ -38,7 +38,6 @@ type ChGoWriter struct { done chan struct{} stopOnce sync.Once - wg sync.WaitGroup poolMetricsDone chan struct{} poolMetricsWG sync.WaitGroup @@ -175,7 +174,21 @@ func (w *ChGoWriter) Stop(_ context.Context) error { w.log.Info("Stopping ch-go writer, flushing remaining buffers") close(w.done) - w.wg.Wait() + + // Drain all table writers. Benthos guarantees all WriteBatch + // goroutines have returned before calling Close/Stop, so no + // in-flight flushes race with this drain. + w.mu.RLock() + + for _, tw := range w.tables { + if err := tw.drainAndFlush(w.config.DrainTimeout); err != nil { + w.log.WithError(err). + WithField("table", tw.table). + Warn("Flush error during shutdown drain") + } + } + + w.mu.RUnlock() if w.poolMetricsDone != nil { close(w.poolMetricsDone) @@ -197,10 +210,6 @@ func (w *ChGoWriter) Write(table string, event *xatu.DecoratedEvent) { select { case tw.buffer <- eventEntry{event: event}: - w.metrics.BufferUsage().WithLabelValues(tw.table).Inc() - w.metrics.BufferUsageTotal().Inc() - - tw.checkBufferWarning() case <-w.done: // Shutting down, discard. } @@ -234,38 +243,23 @@ func (w *ChGoWriter) FlushTables(ctx context.Context, tables []string) error { return nil } - // Send flush requests to matched table writers in parallel. - errChs := make([]chan error, len(writers)) + // Flush each table writer concurrently from the calling goroutines. + errs := make([]error, len(writers)) - for i, tw := range writers { - errCh := make(chan error, 1) - errChs[i] = errCh + var wg sync.WaitGroup - select { - case tw.flushReq <- errCh: - case <-w.done: - return nil - case <-ctx.Done(): - return ctx.Err() - } - } + wg.Add(len(writers)) - // Collect results — return all errors joined. - errs := make([]error, 0, len(errChs)) + for i, tw := range writers { + go func(idx int, tw *chTableWriter) { + defer wg.Done() - for _, errCh := range errChs { - select { - case err := <-errCh: - if err != nil { - errs = append(errs, err) - } - case <-w.done: - return errors.Join(errs...) - case <-ctx.Done(): - errs = append(errs, ctx.Err()) - } + errs[idx] = tw.flushBuffer(ctx) + }(i, tw) } + wg.Wait() + return errors.Join(errs...) } @@ -291,31 +285,23 @@ func (w *ChGoWriter) getOrCreateTableWriter(table string) *chTableWriter { // don't need to include the suffix. cfg := w.config.TableConfigFor(table) tw = &chTableWriter{ - log: w.log.WithField("table", writeTable), - table: writeTable, - baseTable: table, - database: w.database, - config: cfg, - metrics: w.metrics, - writer: w, - buffer: make(chan eventEntry, cfg.BufferSize), - flushReq: make(chan chan error, 1), - organicRetryInitDelay: w.config.OrganicRetryInitDelay, - organicRetryMaxDelay: w.config.OrganicRetryMaxDelay, - drainTimeout: w.config.DrainTimeout, - newBatch: w.batchFactories[table], - limiter: newAdaptiveConcurrencyLimiter(w.chgoCfg.AdaptiveLimiter), + log: w.log.WithField("table", writeTable), + table: writeTable, + baseTable: table, + database: w.database, + config: cfg, + metrics: w.metrics, + writer: w, + buffer: make(chan eventEntry, cfg.BufferSize), + drainTimeout: w.config.DrainTimeout, + newBatch: w.batchFactories[table], + limiter: newAdaptiveConcurrencyLimiter(w.chgoCfg.AdaptiveLimiter), } w.tables[writeTable] = tw - w.wg.Go(func() { - tw.run(w.done) - }) - w.log.WithField("table", writeTable). - WithField("batch_size", cfg.BatchSize). - WithField("flush_interval", cfg.FlushInterval). + WithField("buffer_size", cfg.BufferSize). WithField("has_batch_factory", tw.newBatch != nil). Info("Created ch-go table writer") diff --git a/pkg/consumoor/clickhouse/writer_benchmark_test.go b/pkg/consumoor/clickhouse/writer_benchmark_test.go index b186d12c..9af95e0f 100644 --- a/pkg/consumoor/clickhouse/writer_benchmark_test.go +++ b/pkg/consumoor/clickhouse/writer_benchmark_test.go @@ -79,7 +79,7 @@ func benchHeadEvent() *xatu.DecoratedEvent { // benchWriter creates a ChGoWriter with a noop pool.Do for benchmarking. // The returned cancel function must be called to stop background goroutines. -func benchWriter(tb testing.TB, batchSize, bufferSize int) (*ChGoWriter, context.CancelFunc) { +func benchWriter(tb testing.TB, bufferSize int) (*ChGoWriter, context.CancelFunc) { tb.Helper() ns := fmt.Sprintf("xatu_bench_%d", time.Now().UnixNano()) @@ -94,9 +94,7 @@ func benchWriter(tb testing.TB, batchSize, bufferSize int) (*ChGoWriter, context config: &Config{ DrainTimeout: 30 * time.Second, Defaults: TableConfig{ - BatchSize: batchSize, - FlushInterval: 10 * time.Second, - BufferSize: bufferSize, + BufferSize: bufferSize, }, }, chgoCfg: ChGoConfig{ @@ -120,7 +118,6 @@ func benchWriter(tb testing.TB, batchSize, bufferSize int) (*ChGoWriter, context cancel := func() { w.stopOnce.Do(func() { close(w.done) - w.wg.Wait() }) } @@ -130,12 +127,9 @@ func benchWriter(tb testing.TB, batchSize, bufferSize int) (*ChGoWriter, context // BenchmarkWriteThroughput measures events/sec through the full // Write() -> accumulator -> worker -> noop-flush path. func BenchmarkWriteThroughput(b *testing.B) { - const ( - batchSize = 10_000 - bufferSize = 50_000 - ) + const bufferSize = 50_000 - w, cancel := benchWriter(b, batchSize, bufferSize) + w, cancel := benchWriter(b, bufferSize) defer cancel() event := benchHeadEvent() @@ -162,12 +156,9 @@ func BenchmarkWriteThroughput(b *testing.B) { // BenchmarkWriteConcurrent measures channel contention when multiple // goroutines write to the same ChGoWriter concurrently. func BenchmarkWriteConcurrent(b *testing.B) { - const ( - batchSize = 10_000 - bufferSize = 100_000 - ) + const bufferSize = 100_000 - w, cancel := benchWriter(b, batchSize, bufferSize) + w, cancel := benchWriter(b, bufferSize) defer cancel() event := benchHeadEvent() @@ -196,12 +187,9 @@ func BenchmarkWriteConcurrent(b *testing.B) { // BenchmarkAccumulatorBatching measures how fast the table writer's // run loop drains the buffer channel and builds batches via flush. func BenchmarkAccumulatorBatching(b *testing.B) { - const ( - batchSize = 10_000 - bufferSize = 50_000 - ) + const bufferSize = 50_000 - w, cancel := benchWriter(b, batchSize, bufferSize) + w, cancel := benchWriter(b, bufferSize) defer cancel() event := benchHeadEvent() @@ -236,10 +224,7 @@ func BenchmarkAccumulatorBatching(b *testing.B) { // BenchmarkEndToEndWithFlatten measures the full cost of processing one // event from proto through FlattenTo and the Write path to batch-ready. func BenchmarkEndToEndWithFlatten(b *testing.B) { - const ( - batchSize = 10_000 - bufferSize = 50_000 - ) + const bufferSize = 50_000 var flushCount atomic.Int64 @@ -255,9 +240,7 @@ func BenchmarkEndToEndWithFlatten(b *testing.B) { config: &Config{ DrainTimeout: 30 * time.Second, Defaults: TableConfig{ - BatchSize: batchSize, - FlushInterval: 10 * time.Second, - BufferSize: bufferSize, + BufferSize: bufferSize, }, }, chgoCfg: ChGoConfig{ @@ -308,10 +291,9 @@ func BenchmarkEndToEndWithFlatten(b *testing.B) { b.Logf("FlushTables: %v", err) } - // Shut down the table writer goroutines. + // Signal shutdown. w.stopOnce.Do(func() { close(w.done) - w.wg.Wait() }) elapsed := b.Elapsed().Seconds() @@ -319,3 +301,40 @@ func BenchmarkEndToEndWithFlatten(b *testing.B) { b.ReportMetric(float64(flushCount.Load()), "flushes") b.ReportMetric(float64(b.N)/float64(max(flushCount.Load(), 1)), "events/flush") } + +// BenchmarkConcurrentFlush measures throughput when multiple goroutines +// write and flush concurrently, exercising the concurrent INSERT path. +func BenchmarkConcurrentFlush(b *testing.B) { + const ( + bufferSize = 100_000 + flushers = 8 + batchSize = 500 + ) + + w, cancel := benchWriter(b, bufferSize) + defer cancel() + + event := benchHeadEvent() + + // Pre-create the table writer. + w.Write(benchTable, event) + + b.ReportAllocs() + b.ResetTimer() + + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + // Write a small batch then flush, simulating concurrent WriteBatch calls. + for j := 0; j < batchSize; j++ { + w.Write(benchTable, event) + } + + if err := w.FlushTables(context.Background(), []string{benchTable}); err != nil { + b.Logf("FlushTables: %v", err) + } + } + }) + + b.StopTimer() + b.ReportMetric(float64(b.N*batchSize)/b.Elapsed().Seconds(), "events/sec") +} diff --git a/pkg/consumoor/clickhouse/writer_test.go b/pkg/consumoor/clickhouse/writer_test.go index 98c2dab3..af6cc6d0 100644 --- a/pkg/consumoor/clickhouse/writer_test.go +++ b/pkg/consumoor/clickhouse/writer_test.go @@ -14,6 +14,7 @@ import ( "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/consumoor/telemetry" + "github.com/ethpandaops/xatu/pkg/proto/xatu" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -46,9 +47,7 @@ func newTestWriter(maxRetries int, baseDelay, maxDelay time.Duration) *ChGoWrite DSN: "clickhouse://localhost:9000/default", DrainTimeout: 30 * time.Second, Defaults: TableConfig{ - BatchSize: 100, - FlushInterval: time.Second, - BufferSize: 100, + BufferSize: 100, }, }, metrics: sharedTestMetrics(), @@ -431,207 +430,111 @@ func TestGetOrCreateTableWriter_DifferentTablesGetDifferentWriters(t *testing.T) assert.Len(t, w.tables, 2) } -func TestBufferWarningThresholdValidation(t *testing.T) { - validChGo := ChGoConfig{ - DialTimeout: 5 * time.Second, - ReadTimeout: 30 * time.Second, - RetryBaseDelay: 100 * time.Millisecond, - RetryMaxDelay: 2 * time.Second, - MaxConns: 32, - MinConns: 1, - ConnMaxLifetime: time.Hour, - ConnMaxIdleTime: 10 * time.Minute, - HealthCheckPeriod: 30 * time.Second, +func TestFlushTables_ConcurrentDrain(t *testing.T) { + w := newTestWriter(0, time.Millisecond, time.Millisecond) + w.poolDoFn = func(_ context.Context, _ ch.Query) error { + return nil } - t.Run("accepts 0", func(t *testing.T) { - cfg := &Config{ - DSN: "clickhouse://localhost:9000/default", - Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, - BufferSize: 1000, - }, - OrganicRetryInitDelay: time.Second, - OrganicRetryMaxDelay: 30 * time.Second, - DrainTimeout: 30 * time.Second, - BufferWarningThreshold: 0, - ChGo: validChGo, - } - require.NoError(t, cfg.Validate()) - }) + const table = "beacon_block" - t.Run("accepts 0.8", func(t *testing.T) { - cfg := &Config{ - DSN: "clickhouse://localhost:9000/default", - Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, - BufferSize: 1000, - }, - OrganicRetryInitDelay: time.Second, - OrganicRetryMaxDelay: 30 * time.Second, - DrainTimeout: 30 * time.Second, - BufferWarningThreshold: 0.8, - ChGo: validChGo, - } - require.NoError(t, cfg.Validate()) - }) + // Register a batch factory so flush actually works. + w.batchFactories[table] = func() route.ColumnarBatch { + return &stubBatch{} + } - t.Run("accepts 1", func(t *testing.T) { - cfg := &Config{ - DSN: "clickhouse://localhost:9000/default", - Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, - BufferSize: 1000, - }, - OrganicRetryInitDelay: time.Second, - OrganicRetryMaxDelay: 30 * time.Second, - DrainTimeout: 30 * time.Second, - BufferWarningThreshold: 1.0, - ChGo: validChGo, - } - require.NoError(t, cfg.Validate()) - }) + tw := w.getOrCreateTableWriter(table) - t.Run("rejects negative", func(t *testing.T) { - cfg := &Config{ - DSN: "clickhouse://localhost:9000/default", - Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, - BufferSize: 1000, - }, - OrganicRetryInitDelay: time.Second, - OrganicRetryMaxDelay: 30 * time.Second, - DrainTimeout: 30 * time.Second, - BufferWarningThreshold: -0.1, - ChGo: validChGo, - } - err := cfg.Validate() - require.Error(t, err) - assert.Contains(t, err.Error(), "bufferWarningThreshold") - }) + // Fill the buffer with some events. + const eventCount = 50 - t.Run("rejects greater than 1", func(t *testing.T) { - cfg := &Config{ - DSN: "clickhouse://localhost:9000/default", - Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, - BufferSize: 1000, - }, - OrganicRetryInitDelay: time.Second, - OrganicRetryMaxDelay: 30 * time.Second, - DrainTimeout: 30 * time.Second, - BufferWarningThreshold: 1.5, - ChGo: validChGo, - } - err := cfg.Validate() - require.Error(t, err) - assert.Contains(t, err.Error(), "bufferWarningThreshold") - }) + for i := 0; i < eventCount; i++ { + tw.buffer <- eventEntry{} + } + + // Launch multiple concurrent FlushTables calls. + const flushers = 10 + + errs := make([]error, flushers) + + var wg sync.WaitGroup + + wg.Add(flushers) + + for i := 0; i < flushers; i++ { + go func(idx int) { + defer wg.Done() + + errs[idx] = w.FlushTables(context.Background(), []string{table}) + }(i) + } + + wg.Wait() + + for i, err := range errs { + assert.NoError(t, err, "flusher %d returned error", i) + } + + // Buffer should be fully drained. + assert.Equal(t, 0, len(tw.buffer)) } -func TestCheckBufferWarning(t *testing.T) { - t.Run("no warning below threshold", func(t *testing.T) { - tw := &chTableWriter{ - log: logrus.New().WithField("test", true), - table: "test_table", - config: TableConfig{BufferSize: 100}, - writer: &ChGoWriter{ - config: &Config{BufferWarningThreshold: 0.8}, - }, - buffer: make(chan eventEntry, 100), - } +func TestStop_DrainsRemainingEvents(t *testing.T) { + w := newTestWriter(0, time.Millisecond, time.Millisecond) - // Fill to 50% -- below 80% threshold - for i := 0; i < 50; i++ { - tw.buffer <- eventEntry{} - } + var flushed atomic.Int32 - tw.checkBufferWarning() - assert.Equal(t, int64(0), tw.lastWarnAt.Load(), - "should not warn when below threshold") - }) + w.poolDoFn = func(_ context.Context, _ ch.Query) error { + flushed.Add(1) - t.Run("warns above threshold", func(t *testing.T) { - tw := &chTableWriter{ - log: logrus.New().WithField("test", true), - table: "test_table", - config: TableConfig{BufferSize: 100}, - writer: &ChGoWriter{ - config: &Config{BufferWarningThreshold: 0.8}, - }, - buffer: make(chan eventEntry, 100), - } + return nil + } - // Fill to 85% -- above 80% threshold - for i := 0; i < 85; i++ { - tw.buffer <- eventEntry{} - } + const table = "beacon_block" - tw.checkBufferWarning() - assert.NotEqual(t, int64(0), tw.lastWarnAt.Load(), - "should warn when above threshold") - }) + w.batchFactories[table] = func() route.ColumnarBatch { + return &stubBatch{} + } - t.Run("rate limits warnings", func(t *testing.T) { - tw := &chTableWriter{ - log: logrus.New().WithField("test", true), - table: "test_table", - config: TableConfig{BufferSize: 100}, - writer: &ChGoWriter{ - config: &Config{BufferWarningThreshold: 0.8}, - }, - buffer: make(chan eventEntry, 100), - } + tw := w.getOrCreateTableWriter(table) - // Fill above threshold - for i := 0; i < 85; i++ { - tw.buffer <- eventEntry{} - } + // Buffer some events. + const eventCount = 25 - tw.checkBufferWarning() - firstWarn := tw.lastWarnAt.Load() - require.NotEqual(t, int64(0), firstWarn) + for i := 0; i < eventCount; i++ { + tw.buffer <- eventEntry{} + } - // Second call should not update the timestamp (rate limited) - tw.checkBufferWarning() - assert.Equal(t, firstWarn, tw.lastWarnAt.Load(), - "should rate-limit warnings within the interval") - }) + // Stop should drain and flush remaining events. + err := w.Stop(context.Background()) + require.NoError(t, err) - t.Run("disabled when threshold is 0", func(t *testing.T) { - tw := &chTableWriter{ - log: logrus.New().WithField("test", true), - table: "test_table", - config: TableConfig{BufferSize: 100}, - writer: &ChGoWriter{ - config: &Config{BufferWarningThreshold: 0}, - }, - buffer: make(chan eventEntry, 100), - } + assert.Greater(t, flushed.Load(), int32(0), "should have flushed at least once") + assert.Equal(t, 0, len(tw.buffer), "buffer should be empty after stop") +} - // Fill to 100% - for i := 0; i < 100; i++ { - tw.buffer <- eventEntry{} - } +// stubBatch is a minimal ColumnarBatch implementation for unit tests. +type stubBatch struct { + rows int +} - tw.checkBufferWarning() - assert.Equal(t, int64(0), tw.lastWarnAt.Load(), - "should not warn when threshold is disabled (0)") - }) +func (s *stubBatch) FlattenTo(_ *xatu.DecoratedEvent) error { + s.rows++ + + return nil } +func (s *stubBatch) Rows() int { return s.rows } + +func (s *stubBatch) Input() proto.Input { return proto.Input{} } + +func (s *stubBatch) Reset() { s.rows = 0 } + func TestTableConfigMergeSkipFlattenErrors(t *testing.T) { t.Run("default inherits from defaults", func(t *testing.T) { cfg := &Config{ DSN: "clickhouse://localhost", Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, BufferSize: 1000, SkipFlattenErrors: true, }, @@ -644,9 +547,7 @@ func TestTableConfigMergeSkipFlattenErrors(t *testing.T) { cfg := &Config{ DSN: "clickhouse://localhost", Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, - BufferSize: 1000, + BufferSize: 1000, }, Tables: map[string]TableConfig{ "some_table": {SkipFlattenErrors: true}, @@ -660,8 +561,6 @@ func TestTableConfigMergeSkipFlattenErrors(t *testing.T) { cfg := &Config{ DSN: "clickhouse://localhost", Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, BufferSize: 1000, SkipFlattenErrors: true, }, @@ -677,9 +576,7 @@ func TestTableConfigMergeSkipFlattenErrors(t *testing.T) { cfg := &Config{ DSN: "clickhouse://localhost", Defaults: TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, - BufferSize: 1000, + BufferSize: 1000, }, Tables: map[string]TableConfig{ "some_table": {}, From 7a0b53db9bfa56f3ea3379e7daefc23e45e56618 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 18:04:09 +1000 Subject: [PATCH 07/27] feat(consumoor): add per-topic batch config overrides Allow per-topic overrides for outputBatchCount, outputBatchPeriod, and maxInFlight via kafka.topicOverrides config. Low-volume topics (blocks) can use small batches for low latency while high-volume topics (attestations) use large batches for throughput. --- docs/consumoor-runbook.md | 1 + example_consumoor.yaml | 8 + pkg/consumoor/consumoor.go | 25 +- pkg/consumoor/source/benthos_test.go | 673 ++++++++++++++++++++++++++- pkg/consumoor/source/config.go | 75 ++- pkg/consumoor/source/config_test.go | 89 ++++ 6 files changed, 849 insertions(+), 22 deletions(-) diff --git a/docs/consumoor-runbook.md b/docs/consumoor-runbook.md index 95cd85f1..43984f40 100644 --- a/docs/consumoor-runbook.md +++ b/docs/consumoor-runbook.md @@ -283,6 +283,7 @@ Common causes of startup failure: | `adaptiveLimiter.enabled` | `clickhouse.chgo.adaptiveLimiter.enabled` | true | Per-table AIMD concurrency limiting; disable if all tables have uniform latency | | `adaptiveLimiter.initialLimit` | `clickhouse.chgo.adaptiveLimiter.initialLimit` | 8 | Starting concurrent INSERTs per table before adaptation | | `adaptiveLimiter.maxLimit` | `clickhouse.chgo.adaptiveLimiter.maxLimit` | 50 | Upper bound on concurrent INSERTs per table | +| `topicOverrides` | `kafka.topicOverrides.` | (none) | Per-topic overrides for `outputBatchCount`, `outputBatchPeriod`, and `maxInFlight`; matched by exact topic name | | `tableSuffix` | `clickhouse.tableSuffix` | (empty) | Set to `_local` to bypass Distributed tables in clustered setups | ### Per-Table Overrides diff --git a/example_consumoor.yaml b/example_consumoor.yaml index 25ab787b..0cd946c2 100644 --- a/example_consumoor.yaml +++ b/example_consumoor.yaml @@ -23,6 +23,14 @@ kafka: # outputBatchCount: 1000 # messages per Benthos output batch (0 to disable) # outputBatchPeriod: 1s # max wait before flushing partial batch # maxInFlight: 64 # concurrent WriteBatch calls per stream (higher = more throughput) + # topicOverrides: # per-topic batch settings (exact topic name match) + # general-beacon-block: + # outputBatchCount: 100 + # outputBatchPeriod: 100ms + # maxInFlight: 4 + # general-beacon-attestation: + # outputBatchCount: 50000 + # outputBatchPeriod: 5s # tls: # enabled: false # caFile: /path/to/ca.pem diff --git a/pkg/consumoor/consumoor.go b/pkg/consumoor/consumoor.go index 37225b73..fa147459 100644 --- a/pkg/consumoor/consumoor.go +++ b/pkg/consumoor/consumoor.go @@ -115,9 +115,16 @@ func New( streams := make([]topicStream, 0, len(topics)) for _, topic := range topics { - topicKafkaCfg := config.Kafka + topicKafkaCfg := config.Kafka.ApplyTopicOverride(topic) topicKafkaCfg.Topics = []string{"^" + regexp.QuoteMeta(topic) + "$"} - topicKafkaCfg.ConsumerGroup = config.Kafka.ConsumerGroup + "-" + topic + + if _, hasOverride := config.Kafka.TopicOverrides[topic]; hasOverride { + cLog.WithField("topic", topic). + WithField("outputBatchCount", topicKafkaCfg.OutputBatchCount). + WithField("outputBatchPeriod", topicKafkaCfg.OutputBatchPeriod). + WithField("maxInFlight", topicKafkaCfg.MaxInFlight). + Info("Applied per-topic batch overrides") + } stream, sErr := source.NewBenthosStream( log.WithField("topic", topic), @@ -140,6 +147,8 @@ func New( }) } + metrics.OutputMaxInFlight().Set(float64(config.Kafka.MaxInFlight)) + c := &Consumoor{ log: cLog, config: config, @@ -151,12 +160,12 @@ func New( // Optionally create the Kafka consumer lag monitor. if config.Kafka.LagPollInterval > 0 { - consumerGroups := make([]string, 0, len(topics)) - for _, topic := range topics { - consumerGroups = append(consumerGroups, config.Kafka.ConsumerGroup+"-"+topic) - } - - lagMon, lagErr := source.NewLagMonitor(log, &config.Kafka, consumerGroups, metrics) + lagMon, lagErr := source.NewLagMonitor( + log, + &config.Kafka, + []string{config.Kafka.ConsumerGroup}, + metrics, + ) if lagErr != nil { return nil, fmt.Errorf("creating lag monitor: %w", lagErr) } diff --git a/pkg/consumoor/source/benthos_test.go b/pkg/consumoor/source/benthos_test.go index fbafc196..2171085c 100644 --- a/pkg/consumoor/source/benthos_test.go +++ b/pkg/consumoor/source/benthos_test.go @@ -66,25 +66,20 @@ func (w *testWriter) Stop(context.Context) error { return nil } -func (w *testWriter) Write(table string, _ *xatu.DecoratedEvent) { +func (w *testWriter) FlushTableEvents(_ context.Context, tableEvents map[string][]*xatu.DecoratedEvent) error { + w.flushCalls++ + if w.writes == nil { - w.writes = make(map[string]int) + w.writes = make(map[string]int, len(tableEvents)) } - w.writes[table]++ -} - -func (w *testWriter) FlushTables(_ context.Context, tables []string) error { - w.flushCalls++ - - tableSet := make(map[string]struct{}, len(tables)) - for _, t := range tables { - tableSet[t] = struct{}{} + for table, events := range tableEvents { + w.writes[table] += len(events) } for i, err := range w.flushErrs { if twe, ok := err.(*testWriteError); ok { - if _, found := tableSet[twe.table]; !found { + if _, found := tableEvents[twe.table]; !found { continue } } @@ -583,6 +578,7 @@ func TestKafkaConfig_Validate_OutputBatch(t *testing.T) { ShutdownTimeout: 30 * time.Second, OutputBatchCount: 1000, OutputBatchPeriod: 1 * time.Second, + MaxInFlight: 64, } } @@ -613,6 +609,103 @@ func TestKafkaConfig_Validate_OutputBatch(t *testing.T) { mutate: func(c *KafkaConfig) { c.OutputBatchPeriod = -1 * time.Second }, wantErr: "outputBatchPeriod must be >= 0", }, + { + name: "maxInFlight 1 valid", + mutate: func(c *KafkaConfig) { c.MaxInFlight = 1 }, + }, + { + name: "maxInFlight 0 rejected", + mutate: func(c *KafkaConfig) { c.MaxInFlight = 0 }, + wantErr: "maxInFlight must be >= 1", + }, + { + name: "maxInFlight negative rejected", + mutate: func(c *KafkaConfig) { c.MaxInFlight = -1 }, + wantErr: "maxInFlight must be >= 1", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + cfg := validKafka() + tt.mutate(cfg) + + err := cfg.Validate() + if tt.wantErr != "" { + require.Error(t, err) + assert.Contains(t, err.Error(), tt.wantErr) + } else { + require.NoError(t, err) + } + }) + } +} + +func TestKafkaConfig_Validate_TopicOverrides(t *testing.T) { + validKafka := func() *KafkaConfig { + return &KafkaConfig{ + Brokers: []string{"kafka-1:9092"}, + Topics: []string{"^test-.+"}, + ConsumerGroup: "xatu-consumoor", + Encoding: "json", + FetchMinBytes: 1, + FetchWaitMaxMs: 250, + MaxPartitionFetchBytes: 1048576, + FetchMaxBytes: 10485760, + SessionTimeoutMs: 30000, + OffsetDefault: "earliest", + CommitInterval: 5 * time.Second, + ShutdownTimeout: 30 * time.Second, + OutputBatchCount: 1000, + OutputBatchPeriod: 1 * time.Second, + MaxInFlight: 64, + } + } + + tests := []struct { + name string + mutate func(*KafkaConfig) + wantErr string + }{ + { + name: "valid override passes", + mutate: func(c *KafkaConfig) { + c.TopicOverrides = map[string]TopicOverride{ + "test-blocks": { + OutputBatchCount: intPtr(100), + OutputBatchPeriod: durPtr(100 * time.Millisecond), + MaxInFlight: intPtr(4), + }, + } + }, + }, + { + name: "negative outputBatchCount in override rejected", + mutate: func(c *KafkaConfig) { + c.TopicOverrides = map[string]TopicOverride{ + "test-blocks": {OutputBatchCount: intPtr(-1)}, + } + }, + wantErr: "kafka.topicOverrides.test-blocks: outputBatchCount must be >= 0", + }, + { + name: "negative outputBatchPeriod in override rejected", + mutate: func(c *KafkaConfig) { + c.TopicOverrides = map[string]TopicOverride{ + "test-blocks": {OutputBatchPeriod: durPtr(-1 * time.Second)}, + } + }, + wantErr: "kafka.topicOverrides.test-blocks: outputBatchPeriod must be >= 0", + }, + { + name: "zero maxInFlight in override rejected", + mutate: func(c *KafkaConfig) { + c.TopicOverrides = map[string]TopicOverride{ + "test-blocks": {MaxInFlight: intPtr(0)}, + } + }, + wantErr: "kafka.topicOverrides.test-blocks: maxInFlight must be >= 1", + }, } for _, tt := range tests { @@ -687,6 +780,562 @@ func TestBenthosOutputBatchPolicy(t *testing.T) { } } +// mustNilEventJSON returns a valid JSON-encoded DecoratedEvent with a nil +// Event field. The router treats this as StatusRejected. +func mustNilEventJSON(t *testing.T) []byte { + t.Helper() + + raw, err := protojson.Marshal(&xatu.DecoratedEvent{}) + require.NoError(t, err) + + return raw +} + +// succeededIndexes returns the message indexes that are NOT marked as failed +// in the BatchError. When err is nil, all indexes are returned. +func succeededIndexes(msgs service.MessageBatch, err error) []int { + failed := make(map[int]struct{}, len(msgs)) + + var batchErr *service.BatchError + if errors.As(err, &batchErr) { + //nolint:staticcheck // Tests intentionally use direct indexes from this batch. + batchErr.WalkMessages(func(idx int, _ *service.Message, msgErr error) bool { + if msgErr != nil { + failed[idx] = struct{}{} + } + + return true + }) + } + + out := make([]int, 0, len(msgs)) + + for i := range msgs { + if _, isFailed := failed[i]; !isFailed { + out = append(out, i) + } + } + + return out +} + +// --------------------------------------------------------------------------- +// At-Least-Once Delivery Guarantee Tests +// --------------------------------------------------------------------------- +// +// These tests verify the fundamental at-least-once delivery invariant: +// +// A Kafka offset is committed (message ACK'd) if and only if the message +// has been durably handled: successfully written to ClickHouse, sent to the +// DLQ, or intentionally dropped by the routing layer. +// +// Any other outcome (transient failures, DLQ write errors, context +// cancellation) must cause WriteBatch to return an error, preventing +// Benthos from committing the offset so Kafka redelivers the message. +// +// Tests are organized into three categories: +// +// ack/ — WriteBatch returns nil; all messages are safe to commit. +// nak/ — WriteBatch returns an error; impacted messages must be redelivered. +// mixed/ — Some messages ACK, others NAK within the same batch. + +func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { + headRoute := testRoute{ + eventName: xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD, + table: "beacon_head", + } + blockRoute := testRoute{ + eventName: xatu.Event_BEACON_API_ETH_V1_EVENTS_BLOCK, + table: "beacon_block", + } + + // ----------------------------------------------------------------- + // ACK cases: WriteBatch returns nil — all messages offset-committed. + // ----------------------------------------------------------------- + t.Run("ack", func(t *testing.T) { + t.Run("successful_flush_commits_all_offsets", func(t *testing.T) { + writer := &testWriter{} + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: writer, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{}, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 1), + newKafkaMessage(mustEventJSON(t, "e2", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 2), + newKafkaMessage(mustEventJSON(t, "e3", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 3), + } + + err := output.WriteBatch(context.Background(), msgs) + require.NoError(t, err, "successful flush must ACK all messages") + assert.Equal(t, 1, writer.flushCalls) + assert.Equal(t, 3, writer.writes["beacon_head"]) + }) + + t.Run("decode_error_with_working_dlq_commits_offset", func(t *testing.T) { + rejectSink := &testRejectSink{} + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{}, + metrics: newTestMetrics(), + rejectSink: rejectSink, + } + + msgs := service.MessageBatch{ + newKafkaMessage([]byte("not-valid-json{"), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.NoError(t, err, + "decode error with working DLQ must ACK (message is durably in DLQ)") + require.Len(t, rejectSink.records, 1) + assert.Equal(t, rejectReasonDecode, rejectSink.records[0].Reason) + }) + + t.Run("permanent_write_error_with_working_dlq_commits_offset", func(t *testing.T) { + rejectSink := &testRejectSink{} + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{ + flushErrs: []error{ + &ch.Exception{ + Code: proto.ErrUnknownTable, + Name: "DB::Exception", + Message: "table gone", + }, + }, + }, + metrics: newTestMetrics(), + rejectSink: rejectSink, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.NoError(t, err, + "permanent write error with working DLQ must ACK (message is in DLQ)") + require.Len(t, rejectSink.records, 1) + assert.Equal(t, rejectReasonWritePermanent, rejectSink.records[0].Reason) + }) + + t.Run("unrouted_event_type_silently_commits", func(t *testing.T) { + // Event type has no registered route. The router returns + // StatusDelivered with empty Results, so the message is + // silently skipped — safe to commit. + writer := &testWriter{} + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: writer, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{}, + } + + msgs := service.MessageBatch{ + // BLOCK event has no registered route (only HEAD is registered). + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_BLOCK), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.NoError(t, err, + "unrouted event types are intentionally dropped — safe to commit") + assert.Equal(t, 0, writer.flushCalls, + "no flush should occur for unrouted events") + }) + + t.Run("nil_event_rejected_with_working_dlq_commits", func(t *testing.T) { + // A valid proto with nil Event field triggers StatusRejected + // in the router. With a working DLQ, the message is sent + // there and ACK'd. + rejectSink := &testRejectSink{} + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{}, + metrics: newTestMetrics(), + rejectSink: rejectSink, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustNilEventJSON(t), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.NoError(t, err, + "route-rejected with working DLQ must ACK") + require.Len(t, rejectSink.records, 1) + assert.Equal(t, rejectReasonRouteRejected, rejectSink.records[0].Reason) + }) + + t.Run("nil_event_rejected_without_dlq_still_commits", func(t *testing.T) { + // Route rejections are intentional — the event type is not + // configured. Even without a DLQ the message is ACK'd to + // avoid infinite redelivery of permanently unroutable events. + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{}, + metrics: newTestMetrics(), + rejectSink: nil, // no DLQ + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustNilEventJSON(t), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.NoError(t, err, + "route-rejected without DLQ must still ACK (intentional drop)") + }) + + t.Run("empty_batch_returns_nil", func(t *testing.T) { + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, nil), + writer: &testWriter{}, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{}, + } + + err := output.WriteBatch(context.Background(), service.MessageBatch{}) + require.NoError(t, err) + }) + }) + + // ----------------------------------------------------------------- + // NAK cases: WriteBatch returns error — messages must be redelivered. + // ----------------------------------------------------------------- + t.Run("nak", func(t *testing.T) { + t.Run("transient_flush_error_naks_impacted_messages", func(t *testing.T) { + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{ + flushErrs: []error{ + &testWriteError{table: "beacon_head", permanent: false}, + }, + }, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{}, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 1), + newKafkaMessage(mustEventJSON(t, "e2", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 2), + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err) + assert.Equal(t, []int{0, 1}, failedIndexesFromBatchError(t, msgs, err), + "transient flush failure must NAK all messages in the group") + }) + + t.Run("unknown_flush_error_naks_all_messages", func(t *testing.T) { + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{ + flushErrs: []error{errors.New("unexpected infrastructure error")}, + }, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{}, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err) + assert.Equal(t, []int{0}, failedIndexesFromBatchError(t, msgs, err), + "unclassified flush error must NAK — cannot assume data was written") + }) + + t.Run("context_cancelled_before_processing_naks_batch", func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + cancel() + + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{}, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{}, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 1), + } + + err := output.WriteBatch(ctx, msgs) + require.Error(t, err) + assert.ErrorIs(t, err, context.Canceled, + "cancelled context must NAK the entire batch") + }) + + t.Run("decode_error_without_dlq_naks_for_redelivery", func(t *testing.T) { + // When no DLQ is configured (nil rejectSink), decode errors + // must NAK to avoid silent data loss. The message will be + // redelivered by Kafka indefinitely. + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{}, + metrics: newTestMetrics(), + rejectSink: nil, + } + + msgs := service.MessageBatch{ + newKafkaMessage([]byte("corrupt"), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err) + assert.Equal(t, []int{0}, failedIndexesFromBatchError(t, msgs, err), + "decode error without DLQ must NAK to prevent silent data loss") + }) + + t.Run("permanent_write_error_without_dlq_naks_for_redelivery", func(t *testing.T) { + // Without a DLQ, even permanent write errors must NAK. The + // message will be redelivered, which will fail again — but + // this is preferable to silent data loss. + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{ + flushErrs: []error{ + &ch.Exception{ + Code: proto.ErrUnknownTable, + Name: "DB::Exception", + Message: "table gone", + }, + }, + }, + metrics: newTestMetrics(), + rejectSink: nil, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err) + assert.Equal(t, []int{0}, failedIndexesFromBatchError(t, msgs, err), + "permanent write error without DLQ must NAK") + }) + + t.Run("decode_error_with_broken_dlq_naks", func(t *testing.T) { + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{}, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{ + err: errors.New("kafka produce timeout"), + }, + } + + msgs := service.MessageBatch{ + newKafkaMessage([]byte("{{{"), "t", 0, 1), + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err) + assert.Equal(t, []int{0}, failedIndexesFromBatchError(t, msgs, err), + "DLQ write failure must NAK — message is not durably handled") + }) + + t.Run("permanent_write_error_with_broken_dlq_naks", func(t *testing.T) { + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{ + flushErrs: []error{ + &ch.Exception{ + Code: proto.ErrUnknownTable, + Name: "DB::Exception", + Message: "table gone", + }, + }, + }, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{ + err: errors.New("kafka produce timeout"), + }, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 1), + newKafkaMessage(mustEventJSON(t, "e2", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 2), + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err) + assert.Equal(t, []int{0, 1}, failedIndexesFromBatchError(t, msgs, err), + "permanent error + DLQ failure must NAK all impacted messages") + }) + }) + + // ----------------------------------------------------------------- + // Mixed cases: some messages ACK, others NAK in the same batch. + // ----------------------------------------------------------------- + t.Run("mixed", func(t *testing.T) { + t.Run("valid_and_malformed_with_working_dlq_all_ack", func(t *testing.T) { + // Malformed message goes to DLQ (ACK), valid message is + // flushed to ClickHouse (ACK). Everything commits. + writer := &testWriter{} + rejectSink := &testRejectSink{} + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: writer, + metrics: newTestMetrics(), + rejectSink: rejectSink, + } + + msgs := service.MessageBatch{ + newKafkaMessage([]byte("{bad"), "t", 0, 1), + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 2), + } + + err := output.WriteBatch(context.Background(), msgs) + require.NoError(t, err, + "malformed→DLQ + valid→CH: both are durably handled, all ACK") + assert.Equal(t, 1, writer.flushCalls) + assert.Equal(t, 1, writer.writes["beacon_head"]) + require.Len(t, rejectSink.records, 1) + assert.Equal(t, rejectReasonDecode, rejectSink.records[0].Reason) + }) + + t.Run("two_event_groups_one_transient_fail_only_that_group_naks", func(t *testing.T) { + // Two event types in the same batch. One group flushes + // successfully, the other hits a transient error. Only the + // failed group's messages should be NAK'd. + writer := &testWriter{ + flushErrs: []error{ + &testWriteError{table: "beacon_block", permanent: false}, + }, + } + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute, blockRoute}), + writer: writer, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{}, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 1), + newKafkaMessage(mustEventJSON(t, "e2", xatu.Event_BEACON_API_ETH_V1_EVENTS_BLOCK), "t", 0, 2), + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err, "batch should fail because one group had a transient error") + + failed := failedIndexesFromBatchError(t, msgs, err) + succeeded := succeededIndexes(msgs, err) + + // HEAD (index 0) should succeed, BLOCK (index 1) should fail. + assert.Equal(t, []int{1}, failed, + "only the BLOCK group (transient failure) should be NAK'd") + assert.Equal(t, []int{0}, succeeded, + "the HEAD group (successful flush) should be ACK'd") + }) + + t.Run("malformed_with_broken_dlq_naks_valid_messages_still_ack", func(t *testing.T) { + // Decode error + DLQ failure → NAK that message. + // Valid message → successful flush → ACK. + writer := &testWriter{} + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: writer, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{ + err: errors.New("dlq unavailable"), + }, + } + + msgs := service.MessageBatch{ + newKafkaMessage([]byte("{bad"), "t", 0, 1), + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 2), + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err) + + failed := failedIndexesFromBatchError(t, msgs, err) + succeeded := succeededIndexes(msgs, err) + + assert.Equal(t, []int{0}, failed, + "malformed message with DLQ failure must NAK") + assert.Equal(t, []int{1}, succeeded, + "valid message with successful flush must ACK") + }) + + t.Run("decode_dlq_failure_plus_flush_failure_naks_both", func(t *testing.T) { + // Edge case: Phase 1 produces a NAK (decode error + DLQ + // failure), AND Phase 2 produces a NAK (transient flush). + // Both messages should be NAK'd. + // + // NOTE: WriteBatch currently overwrites the Phase 1 + // BatchError when Phase 2 fails, losing the decode + // failure's NAK. This test documents the expected correct + // behavior — if it fails, it reveals a bug where Phase 1 + // failures are silently dropped. + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{ + flushErrs: []error{ + &testWriteError{table: "beacon_head", permanent: false}, + }, + }, + metrics: newTestMetrics(), + rejectSink: &testRejectSink{ + err: errors.New("dlq unavailable"), + }, + } + + msgs := service.MessageBatch{ + newKafkaMessage([]byte("{bad"), "t", 0, 1), // decode error → DLQ fails + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD), "t", 0, 2), // flush fails + } + + err := output.WriteBatch(context.Background(), msgs) + require.Error(t, err) + assert.Equal(t, []int{0, 1}, failedIndexesFromBatchError(t, msgs, err), + "both the decode-DLQ-failed message and flush-failed message must NAK") + }) + }) +} + func TestFranzSASLMechanism(t *testing.T) { t.Run("defaults to PLAIN", func(t *testing.T) { mech, err := franzSASLMechanism(&SASLConfig{ diff --git a/pkg/consumoor/source/config.go b/pkg/consumoor/source/config.go index 6f63ef6e..3c86cc01 100644 --- a/pkg/consumoor/source/config.go +++ b/pkg/consumoor/source/config.go @@ -26,6 +26,31 @@ var supportedSASLMechanisms = map[string]struct{}{ SASLMechanismOAUTHBEARER: {}, } +// TopicOverride holds per-topic batch settings that override KafkaConfig defaults. +// Nil pointer fields inherit the global default. +type TopicOverride struct { + OutputBatchCount *int `yaml:"outputBatchCount"` + OutputBatchPeriod *time.Duration `yaml:"outputBatchPeriod"` + MaxInFlight *int `yaml:"maxInFlight"` +} + +// Validate checks the per-topic override for errors. +func (o *TopicOverride) Validate(topic string) error { + if o.OutputBatchCount != nil && *o.OutputBatchCount < 0 { + return fmt.Errorf("kafka.topicOverrides.%s: outputBatchCount must be >= 0", topic) + } + + if o.OutputBatchPeriod != nil && *o.OutputBatchPeriod < 0 { + return fmt.Errorf("kafka.topicOverrides.%s: outputBatchPeriod must be >= 0", topic) + } + + if o.MaxInFlight != nil && *o.MaxInFlight < 1 { + return fmt.Errorf("kafka.topicOverrides.%s: maxInFlight must be >= 1", topic) + } + + return nil +} + // KafkaConfig configures the Kafka consumer. type KafkaConfig struct { // Brokers is a list of Kafka broker addresses. @@ -86,12 +111,23 @@ type KafkaConfig struct { // OutputBatchCount is the number of messages Benthos accumulates before // calling WriteBatch on the output plugin. Higher values increase INSERT // throughput by writing more rows per ClickHouse INSERT. Set to 0 to - // disable count-based batching. Default: 1000. - OutputBatchCount int `yaml:"outputBatchCount" default:"1000"` + // disable count-based batching. Default: 10000. + OutputBatchCount int `yaml:"outputBatchCount" default:"10000"` // OutputBatchPeriod is the maximum time Benthos waits to fill a batch // before flushing a partial batch. Ensures low-volume topics still make // progress. Default: 1s. Set to 0 to disable period-based flushing. OutputBatchPeriod time.Duration `yaml:"outputBatchPeriod" default:"1s"` + + // MaxInFlight is the maximum number of concurrent WriteBatch calls + // Benthos makes for each stream's output. Higher values increase + // throughput by allowing concurrent ClickHouse INSERTs and bigger + // natural batches. Default: 8. + MaxInFlight int `yaml:"maxInFlight" default:"64"` + + // TopicOverrides contains per-topic batch settings keyed by exact topic name. + // Overrides are matched against discovered concrete topic names. Unset fields + // inherit the global defaults from this KafkaConfig. + TopicOverrides map[string]TopicOverride `yaml:"topicOverrides"` } // SASLConfig configures SASL authentication for Kafka. @@ -154,6 +190,16 @@ func (c *KafkaConfig) Validate() error { return errors.New("kafka: outputBatchPeriod must be >= 0") } + if c.MaxInFlight < 1 { + return errors.New("kafka: maxInFlight must be >= 1") + } + + for topic, override := range c.TopicOverrides { + if err := override.Validate(topic); err != nil { + return err + } + } + if err := c.TLS.Validate(); err != nil { return fmt.Errorf("kafka.%w", err) } @@ -167,6 +213,31 @@ func (c *KafkaConfig) Validate() error { return nil } +// ApplyTopicOverride returns a shallow copy with per-topic overrides merged in. +// Fields not set in the override keep the global default. +func (c *KafkaConfig) ApplyTopicOverride(topic string) KafkaConfig { + out := *c + + override, ok := c.TopicOverrides[topic] + if !ok { + return out + } + + if override.OutputBatchCount != nil { + out.OutputBatchCount = *override.OutputBatchCount + } + + if override.OutputBatchPeriod != nil { + out.OutputBatchPeriod = *override.OutputBatchPeriod + } + + if override.MaxInFlight != nil { + out.MaxInFlight = *override.MaxInFlight + } + + return out +} + // heartbeatIntervalMs derives the heartbeat interval from the session timeout. // Kafka's standard practice is sessionTimeout / 10. func (c *KafkaConfig) heartbeatIntervalMs() int { diff --git a/pkg/consumoor/source/config_test.go b/pkg/consumoor/source/config_test.go index 0499f0c6..91b08cb0 100644 --- a/pkg/consumoor/source/config_test.go +++ b/pkg/consumoor/source/config_test.go @@ -18,6 +18,95 @@ func validKafkaConfig() KafkaConfig { SessionTimeoutMs: 30000, CommitInterval: 5 * time.Second, ShutdownTimeout: 30 * time.Second, + MaxInFlight: 64, + } +} + +func intPtr(v int) *int { return &v } +func durPtr(v time.Duration) *time.Duration { return &v } + +func TestKafkaConfig_ApplyTopicOverride(t *testing.T) { + tests := []struct { + name string + overrides map[string]TopicOverride + topic string + wantCount int + wantPeriod time.Duration + wantFlight int + }{ + { + name: "no override returns unchanged copy", + overrides: nil, + topic: "general-beacon-block", + wantCount: 10000, + wantPeriod: 1 * time.Second, + wantFlight: 64, + }, + { + name: "partial override (only count)", + overrides: map[string]TopicOverride{ + "general-beacon-block": { + OutputBatchCount: intPtr(100), + }, + }, + topic: "general-beacon-block", + wantCount: 100, + wantPeriod: 1 * time.Second, + wantFlight: 64, + }, + { + name: "full override", + overrides: map[string]TopicOverride{ + "general-beacon-block": { + OutputBatchCount: intPtr(500), + OutputBatchPeriod: durPtr(200 * time.Millisecond), + MaxInFlight: intPtr(4), + }, + }, + topic: "general-beacon-block", + wantCount: 500, + wantPeriod: 200 * time.Millisecond, + wantFlight: 4, + }, + { + name: "explicit zero count", + overrides: map[string]TopicOverride{ + "general-beacon-block": { + OutputBatchCount: intPtr(0), + }, + }, + topic: "general-beacon-block", + wantCount: 0, + wantPeriod: 1 * time.Second, + wantFlight: 64, + }, + { + name: "unmatched topic returns unchanged copy", + overrides: map[string]TopicOverride{ + "general-beacon-attestation": { + OutputBatchCount: intPtr(50000), + }, + }, + topic: "general-beacon-block", + wantCount: 10000, + wantPeriod: 1 * time.Second, + wantFlight: 64, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + cfg := validKafkaConfig() + cfg.OutputBatchCount = 10000 + cfg.OutputBatchPeriod = 1 * time.Second + cfg.TopicOverrides = tt.overrides + + got := cfg.ApplyTopicOverride(tt.topic) + + assert.Equal(t, tt.wantCount, got.OutputBatchCount) + assert.Equal(t, tt.wantPeriod, got.OutputBatchPeriod) + assert.Equal(t, tt.wantFlight, got.MaxInFlight) + }) } } From e974d8cc3a11de333dd35d4310cacb4497779a5e Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 18:46:12 +1000 Subject: [PATCH 08/27] perf(consumoor): reduce GC pressure via allocation optimizations MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit CPU profiling showed GC consuming ~32% of CPU. This commit reduces per-event allocations in the hot path: - Parse consensus version once per event instead of 7 SplitN calls across 4 columns (updated codegen + regenerated all .gen.go files) - Replace fmt.Sprint/Sprintf with strconv.FormatUint for uint64→string - Cache INSERT query string and operation name per table writer - Avoid defensive byte copy on success path (defer to DLQ path only) - Hoist protojson.UnmarshalOptions to package-level var - Short-circuit table+suffix concat when suffix is empty - Remove stale BufferSize/DrainTimeout fields and fix broken tests --- pkg/consumoor/clickhouse/config.go | 25 --- pkg/consumoor/clickhouse/table_writer.go | 110 +++------ pkg/consumoor/clickhouse/writer.go | 115 ++++------ .../clickhouse/writer_benchmark_test.go | 211 +++++------------- pkg/consumoor/clickhouse/writer_test.go | 79 ++----- pkg/consumoor/config_test.go | 62 +---- pkg/consumoor/health_test.go | 12 +- .../beacon_api_eth_v1_beacon_blob.gen.go | 9 +- .../beacon_api_eth_v1_beacon_committee.gen.go | 9 +- .../beacon_api_eth_v1_beacon_committee.go | 3 +- ...eacon_api_eth_v1_events_attestation.gen.go | 9 +- .../beacon_api_eth_v1_events_attestation.go | 5 +- ...acon_api_eth_v1_events_blob_sidecar.gen.go | 9 +- .../beacon_api_eth_v1_events_block.gen.go | 9 +- ...acon_api_eth_v1_events_block_gossip.gen.go | 9 +- ...eacon_api_eth_v1_events_chain_reorg.gen.go | 9 +- ...th_v1_events_contribution_and_proof.gen.go | 9 +- ...pi_eth_v1_events_contribution_and_proof.go | 3 +- ...i_eth_v1_events_data_column_sidecar.gen.go | 9 +- ..._eth_v1_events_finalized_checkpoint.gen.go | 9 +- .../beacon_api_eth_v1_events_head.gen.go | 9 +- ...on_api_eth_v1_events_voluntary_exit.gen.go | 9 +- .../beacon_api_eth_v1_proposer_duty.gen.go | 9 +- ...i_eth_v1_validator_attestation_data.gen.go | 9 +- ...n_api_eth_v1_validator_attestation_data.go | 3 +- .../beacon_api_eth_v2_beacon_block.gen.go | 9 +- .../beacon_api_eth_v3_validator_block.gen.go | 9 +- .../canonical_beacon_blob_sidecar.gen.go | 9 +- .../canonical/canonical_beacon_block.gen.go | 9 +- ...ical_beacon_block_attester_slashing.gen.go | 9 +- ...eacon_block_bls_to_execution_change.gen.go | 9 +- .../canonical_beacon_block_deposit.gen.go | 9 +- .../canonical_beacon_block_deposit.go | 3 +- ..._beacon_block_execution_transaction.gen.go | 9 +- ...ical_beacon_block_proposer_slashing.gen.go | 9 +- ...nonical_beacon_block_sync_aggregate.gen.go | 9 +- ...nonical_beacon_block_voluntary_exit.gen.go | 9 +- .../canonical_beacon_block_withdrawal.gen.go | 9 +- .../canonical_beacon_block_withdrawal.go | 3 +- .../canonical_beacon_committee.gen.go | 9 +- .../canonical/canonical_beacon_committee.go | 3 +- ...nical_beacon_elaborated_attestation.gen.go | 9 +- ...canonical_beacon_elaborated_attestation.go | 4 +- .../canonical_beacon_proposer_duty.gen.go | 9 +- .../canonical_beacon_sync_committee.gen.go | 9 +- .../canonical_beacon_validators.gen.go | 9 +- ...canonical_beacon_validators_pubkeys.gen.go | 9 +- ...n_validators_withdrawal_credentials.gen.go | 9 +- pkg/consumoor/route/cmd/chgo-rowgen/main.go | 55 ++++- pkg/consumoor/route/column.go | 19 +- pkg/consumoor/source/benthos.go | 2 +- pkg/consumoor/source/kafka.go | 5 +- pkg/consumoor/source/lag_monitor.go | 6 +- pkg/consumoor/source/output.go | 37 ++- pkg/consumoor/source/writer.go | 9 +- pkg/consumoor/telemetry/metrics.go | 55 ++--- 56 files changed, 453 insertions(+), 673 deletions(-) diff --git a/pkg/consumoor/clickhouse/config.go b/pkg/consumoor/clickhouse/config.go index cbeebd5e..571a2bad 100644 --- a/pkg/consumoor/clickhouse/config.go +++ b/pkg/consumoor/clickhouse/config.go @@ -31,11 +31,6 @@ type Config struct { // false to downgrade to warnings and allow startup to proceed. FailOnMissingTables bool `yaml:"failOnMissingTables" default:"true"` - // DrainTimeout bounds how long each table writer waits for its final - // flush during shutdown. If ClickHouse is unresponsive the drain is - // cancelled after this duration rather than hanging indefinitely. - DrainTimeout time.Duration `yaml:"drainTimeout" default:"30s"` - // Defaults are the default table settings. Defaults TableConfig `yaml:"defaults"` @@ -48,10 +43,6 @@ type Config struct { // TableConfig holds per-table settings for the ClickHouse writer. type TableConfig struct { - // BufferSize is the channel buffer capacity for pending rows. - // When the buffer is full, Write() blocks, propagating backpressure - // through Benthos to the Kafka consumer. - BufferSize int `yaml:"bufferSize" default:"50000"` // SkipFlattenErrors when true skips events that fail FlattenTo // instead of failing the entire batch. Default false = fail-fast. SkipFlattenErrors bool `yaml:"skipFlattenErrors"` @@ -177,23 +168,11 @@ func (c *Config) Validate() error { return err } - if c.DrainTimeout <= 0 { - return errors.New("clickhouse: drainTimeout must be > 0") - } - - if c.Defaults.BufferSize <= 0 { - return errors.New("clickhouse.defaults: bufferSize must be > 0") - } - if err := validateInsertSettings(c.Defaults.InsertSettings, "clickhouse.defaults.insertSettings"); err != nil { return err } for table, override := range c.Tables { - if override.BufferSize < 0 { - return fmt.Errorf("clickhouse.tables.%s: bufferSize must be >= 0", table) - } - path := fmt.Sprintf("clickhouse.tables.%s.insertSettings", table) if err := validateInsertSettings(override.InsertSettings, path); err != nil { return err @@ -277,10 +256,6 @@ func (c *Config) TableConfigFor(table string) TableConfig { return cfg } - if override.BufferSize > 0 { - cfg.BufferSize = override.BufferSize - } - cfg.SkipFlattenErrors = cfg.SkipFlattenErrors || override.SkipFlattenErrors if len(override.InsertSettings) > 0 { diff --git a/pkg/consumoor/clickhouse/table_writer.go b/pkg/consumoor/clickhouse/table_writer.go index 34aa8620..c0b23233 100644 --- a/pkg/consumoor/clickhouse/table_writer.go +++ b/pkg/consumoor/clickhouse/table_writer.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "sync" "time" "github.com/ClickHouse/ch-go" @@ -14,11 +13,6 @@ import ( "github.com/sirupsen/logrus" ) -// eventEntry holds a single event for buffering in the table writer. -type eventEntry struct { - event *xatu.DecoratedEvent -} - type chTableWriter struct { log logrus.FieldLogger table string @@ -27,80 +21,20 @@ type chTableWriter struct { config TableConfig metrics *telemetry.Metrics writer *ChGoWriter - buffer chan eventEntry - - drainTimeout time.Duration - - // drainMu serializes buffer drains so that the first goroutine gets a - // large batch while subsequent concurrent flushers find an empty buffer. - drainMu sync.Mutex newBatch func() route.ColumnarBatch // limiter is the per-table adaptive concurrency limiter. // nil when adaptive limiting is disabled. limiter *adaptiveConcurrencyLimiter -} - -// flushBuffer drains the buffer channel and flushes all accumulated events. -// The drain is serialized via drainMu so that the first caller gets a large -// batch while concurrent callers find an empty buffer and return nil. -func (tw *chTableWriter) flushBuffer(ctx context.Context) error { - tw.drainMu.Lock() - - events := make([]eventEntry, 0, len(tw.buffer)) - - for { - select { - case entry := <-tw.buffer: - events = append(events, entry) - default: - goto drained - } - } - -drained: - - tw.drainMu.Unlock() - - if len(events) == 0 { - return nil - } - - return tw.flush(ctx, events) -} - -// drainAndFlush drains the buffer and flushes with a timeout context. -// Used during shutdown to flush any remaining events. -func (tw *chTableWriter) drainAndFlush(timeout time.Duration) error { - tw.drainMu.Lock() - - events := make([]eventEntry, 0, len(tw.buffer)) - for { - select { - case entry := <-tw.buffer: - events = append(events, entry) - default: - goto drained - } - } - -drained: - - tw.drainMu.Unlock() - - if len(events) == 0 { - return nil - } - - ctx, cancel := context.WithTimeout(context.Background(), timeout) - defer cancel() - - return tw.flush(ctx, events) + // Cached per-table strings computed once and reused every flush. + operationName string + insertQuery string + insertQueryOK bool } -func (tw *chTableWriter) flush(ctx context.Context, events []eventEntry) error { +func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEvent) error { if len(events) == 0 { return nil } @@ -125,8 +59,8 @@ func (tw *chTableWriter) flush(ctx context.Context, events []eventEntry) error { lastErr error ) - for _, e := range events { - err := batch.FlattenTo(e.event) + for _, event := range events { + err := batch.FlattenTo(event) if err == nil { continue } @@ -178,21 +112,29 @@ func (tw *chTableWriter) flush(ctx context.Context, events []eventEntry) error { input := batch.Input() - insertBody, err := insertQueryWithSettings(input.Into(tw.table), tw.config.InsertSettings) - if err != nil { - tw.log.WithError(err). - WithField("rows", rows). - Error("Invalid insert settings") - tw.metrics.WriteErrors().WithLabelValues(tw.table).Add(float64(rows)) + // Cache the INSERT query body and operation name on first use. + // Both are invariant between flushes for a given table. + if !tw.insertQueryOK { + body, err := insertQueryWithSettings(input.Into(tw.table), tw.config.InsertSettings) + if err != nil { + tw.log.WithError(err). + WithField("rows", rows). + Error("Invalid insert settings") + tw.metrics.WriteErrors().WithLabelValues(tw.table).Add(float64(rows)) - return &tableWriteError{ - table: tw.baseTable, - cause: fmt.Errorf("building insert query for %s: %w", tw.table, err), + return &tableWriteError{ + table: tw.baseTable, + cause: fmt.Errorf("building insert query for %s: %w", tw.table, err), + } } + + tw.insertQuery = body + tw.operationName = "insert_" + tw.table + tw.insertQueryOK = true } - if err := tw.do(ctx, "insert_"+tw.table, &ch.Query{ - Body: insertBody, + if err := tw.do(ctx, tw.operationName, &ch.Query{ + Body: tw.insertQuery, Input: input, }, nil); err != nil { tw.log.WithError(err). diff --git a/pkg/consumoor/clickhouse/writer.go b/pkg/consumoor/clickhouse/writer.go index 9fe1b746..08f090bd 100644 --- a/pkg/consumoor/clickhouse/writer.go +++ b/pkg/consumoor/clickhouse/writer.go @@ -36,7 +36,6 @@ type ChGoWriter struct { // registered by route initialization. batchFactories map[string]func() route.ColumnarBatch - done chan struct{} stopOnce sync.Once poolMetricsDone chan struct{} @@ -77,7 +76,6 @@ func NewChGoWriter( database: opts.Database, tables: make(map[string]*chTableWriter, 16), batchFactories: make(map[string]func() route.ColumnarBatch, 64), - done: make(chan struct{}), }, nil } @@ -167,28 +165,11 @@ func (w *ChGoWriter) Ping(ctx context.Context) error { return pool.Ping(ctx) } -// Stop drains buffers and closes the connection pool. It is safe to -// call multiple times; only the first call performs cleanup. +// Stop closes the connection pool. It is safe to call multiple times; +// only the first call performs cleanup. func (w *ChGoWriter) Stop(_ context.Context) error { w.stopOnce.Do(func() { - w.log.Info("Stopping ch-go writer, flushing remaining buffers") - - close(w.done) - - // Drain all table writers. Benthos guarantees all WriteBatch - // goroutines have returned before calling Close/Stop, so no - // in-flight flushes race with this drain. - w.mu.RLock() - - for _, tw := range w.tables { - if err := tw.drainAndFlush(w.config.DrainTimeout); err != nil { - w.log.WithError(err). - WithField("table", tw.table). - Warn("Flush error during shutdown drain") - } - } - - w.mu.RUnlock() + w.log.Info("Stopping ch-go writer") if w.poolMetricsDone != nil { close(w.poolMetricsDone) @@ -203,59 +184,49 @@ func (w *ChGoWriter) Stop(_ context.Context) error { return nil } -// Write enqueues an event for table batching. -// Write blocks if the buffer is full, propagating backpressure to the caller. -func (w *ChGoWriter) Write(table string, event *xatu.DecoratedEvent) { - tw := w.getOrCreateTableWriter(table) - - select { - case tw.buffer <- eventEntry{event: event}: - case <-w.done: - // Shutting down, discard. - } -} - -// FlushTables forces the specified table writers (by base table name) -// to drain their buffers and write to ClickHouse synchronously. -// Base names are resolved using the configured TableSuffix. -// An empty or nil slice is a no-op that returns nil. -// Returns a joined error containing all table failures so callers can -// identify every failed table in the batch. -func (w *ChGoWriter) FlushTables(ctx context.Context, tables []string) error { - if len(tables) == 0 { +// FlushTableEvents writes the given events directly to their respective +// ClickHouse tables concurrently. The map keys are base table names +// (without suffix). Returns a joined error containing all table failures. +func (w *ChGoWriter) FlushTableEvents( + ctx context.Context, + tableEvents map[string][]*xatu.DecoratedEvent, +) error { + if len(tableEvents) == 0 { return nil } - w.mu.RLock() + type tableFlush struct { + tw *chTableWriter + events []*xatu.DecoratedEvent + } - writers := make([]*chTableWriter, 0, len(tables)) + flushes := make([]tableFlush, 0, len(tableEvents)) - for _, base := range tables { - writeTable := base + w.config.TableSuffix - if tw, ok := w.tables[writeTable]; ok { - writers = append(writers, tw) + for base, events := range tableEvents { + if len(events) == 0 { + continue } - } - w.mu.RUnlock() + tw := w.getOrCreateTableWriter(base) + flushes = append(flushes, tableFlush{tw: tw, events: events}) + } - if len(writers) == 0 { + if len(flushes) == 0 { return nil } - // Flush each table writer concurrently from the calling goroutines. - errs := make([]error, len(writers)) + errs := make([]error, len(flushes)) var wg sync.WaitGroup - wg.Add(len(writers)) + wg.Add(len(flushes)) - for i, tw := range writers { - go func(idx int, tw *chTableWriter) { + for i, f := range flushes { + go func(idx int, f tableFlush) { defer wg.Done() - errs[idx] = tw.flushBuffer(ctx) - }(i, tw) + errs[idx] = f.tw.flush(ctx, f.events) + }(i, f) } wg.Wait() @@ -264,7 +235,12 @@ func (w *ChGoWriter) FlushTables(ctx context.Context, tables []string) error { } func (w *ChGoWriter) getOrCreateTableWriter(table string) *chTableWriter { - writeTable := table + w.config.TableSuffix + var writeTable string + if w.config.TableSuffix == "" { + writeTable = table + } else { + writeTable = table + w.config.TableSuffix + } w.mu.RLock() tw, ok := w.tables[writeTable] @@ -285,23 +261,20 @@ func (w *ChGoWriter) getOrCreateTableWriter(table string) *chTableWriter { // don't need to include the suffix. cfg := w.config.TableConfigFor(table) tw = &chTableWriter{ - log: w.log.WithField("table", writeTable), - table: writeTable, - baseTable: table, - database: w.database, - config: cfg, - metrics: w.metrics, - writer: w, - buffer: make(chan eventEntry, cfg.BufferSize), - drainTimeout: w.config.DrainTimeout, - newBatch: w.batchFactories[table], - limiter: newAdaptiveConcurrencyLimiter(w.chgoCfg.AdaptiveLimiter), + log: w.log.WithField("table", writeTable), + table: writeTable, + baseTable: table, + database: w.database, + config: cfg, + metrics: w.metrics, + writer: w, + newBatch: w.batchFactories[table], + limiter: newAdaptiveConcurrencyLimiter(w.chgoCfg.AdaptiveLimiter), } w.tables[writeTable] = tw w.log.WithField("table", writeTable). - WithField("buffer_size", cfg.BufferSize). WithField("has_batch_factory", tw.newBatch != nil). Info("Created ch-go table writer") diff --git a/pkg/consumoor/clickhouse/writer_benchmark_test.go b/pkg/consumoor/clickhouse/writer_benchmark_test.go index 9af95e0f..de0e5abb 100644 --- a/pkg/consumoor/clickhouse/writer_benchmark_test.go +++ b/pkg/consumoor/clickhouse/writer_benchmark_test.go @@ -3,7 +3,6 @@ package clickhouse import ( "context" "fmt" - "sync" "sync/atomic" "testing" "time" @@ -78,8 +77,7 @@ func benchHeadEvent() *xatu.DecoratedEvent { } // benchWriter creates a ChGoWriter with a noop pool.Do for benchmarking. -// The returned cancel function must be called to stop background goroutines. -func benchWriter(tb testing.TB, bufferSize int) (*ChGoWriter, context.CancelFunc) { +func benchWriter(tb testing.TB) *ChGoWriter { tb.Helper() ns := fmt.Sprintf("xatu_bench_%d", time.Now().UnixNano()) @@ -91,12 +89,7 @@ func benchWriter(tb testing.TB, bufferSize int) (*ChGoWriter, context.CancelFunc w := &ChGoWriter{ log: log.WithField("component", "bench"), metrics: metrics, - config: &Config{ - DrainTimeout: 30 * time.Second, - Defaults: TableConfig{ - BufferSize: bufferSize, - }, - }, + config: &Config{}, chgoCfg: ChGoConfig{ MaxRetries: 0, RetryBaseDelay: 100 * time.Millisecond, @@ -104,7 +97,6 @@ func benchWriter(tb testing.TB, bufferSize int) (*ChGoWriter, context.CancelFunc }, tables: make(map[string]*chTableWriter, 4), batchFactories: make(map[string]func() route.ColumnarBatch, 4), - done: make(chan struct{}), poolDoFn: func(_ context.Context, _ ch.Query) error { return nil }, @@ -115,117 +107,97 @@ func benchWriter(tb testing.TB, bufferSize int) (*ChGoWriter, context.CancelFunc w.RegisterBatchFactories(routes) - cancel := func() { - w.stopOnce.Do(func() { - close(w.done) - }) - } - - return w, cancel + return w } -// BenchmarkWriteThroughput measures events/sec through the full -// Write() -> accumulator -> worker -> noop-flush path. -func BenchmarkWriteThroughput(b *testing.B) { - const bufferSize = 50_000 - - w, cancel := benchWriter(b, bufferSize) - defer cancel() - +// BenchmarkFlushTableEvents measures throughput of the FlushTableEvents path +// (FlattenTo + columnar batch build + noop INSERT). +func BenchmarkFlushTableEvents(b *testing.B) { + w := benchWriter(b) event := benchHeadEvent() b.ReportAllocs() b.ResetTimer() for i := 0; i < b.N; i++ { - w.Write(benchTable, event) - } - - b.StopTimer() - - ctx, ctxCancel := context.WithTimeout(context.Background(), 10*time.Second) - defer ctxCancel() + tableEvents := map[string][]*xatu.DecoratedEvent{ + benchTable: {event}, + } - if err := w.FlushTables(ctx, []string{benchTable}); err != nil { - b.Logf("FlushTables: %v", err) + if err := w.FlushTableEvents(context.Background(), tableEvents); err != nil { + b.Fatalf("FlushTableEvents: %v", err) + } } + b.StopTimer() b.ReportMetric(float64(b.N)/b.Elapsed().Seconds(), "events/sec") } -// BenchmarkWriteConcurrent measures channel contention when multiple -// goroutines write to the same ChGoWriter concurrently. -func BenchmarkWriteConcurrent(b *testing.B) { - const bufferSize = 100_000 - - w, cancel := benchWriter(b, bufferSize) - defer cancel() +// BenchmarkFlushTableEventsBatch measures throughput with larger batches. +func BenchmarkFlushTableEventsBatch(b *testing.B) { + const batchSize = 500 + w := benchWriter(b) event := benchHeadEvent() + // Pre-build the batch of events. + events := make([]*xatu.DecoratedEvent, batchSize) + for i := range events { + events[i] = event + } + b.ReportAllocs() b.ResetTimer() - b.RunParallel(func(pb *testing.PB) { - for pb.Next() { - w.Write(benchTable, event) + for i := 0; i < b.N; i++ { + tableEvents := map[string][]*xatu.DecoratedEvent{ + benchTable: events, } - }) - b.StopTimer() - - ctx, ctxCancel := context.WithTimeout(context.Background(), 10*time.Second) - defer ctxCancel() - - if err := w.FlushTables(ctx, []string{benchTable}); err != nil { - b.Logf("FlushTables: %v", err) + if err := w.FlushTableEvents(context.Background(), tableEvents); err != nil { + b.Fatalf("FlushTableEvents: %v", err) + } } - b.ReportMetric(float64(b.N)/b.Elapsed().Seconds(), "events/sec") + b.StopTimer() + b.ReportMetric(float64(b.N*batchSize)/b.Elapsed().Seconds(), "events/sec") } -// BenchmarkAccumulatorBatching measures how fast the table writer's -// run loop drains the buffer channel and builds batches via flush. -func BenchmarkAccumulatorBatching(b *testing.B) { - const bufferSize = 50_000 - - w, cancel := benchWriter(b, bufferSize) - defer cancel() +// BenchmarkFlushConcurrent measures throughput when multiple goroutines +// flush concurrently, exercising the concurrent INSERT path. +func BenchmarkFlushConcurrent(b *testing.B) { + const batchSize = 500 + w := benchWriter(b) event := benchHeadEvent() - // Pre-populate the buffer to measure drain/flush speed. - // First trigger table writer creation by writing one event. - w.Write(benchTable, event) - - // Wait briefly for the table writer goroutine to start. - time.Sleep(10 * time.Millisecond) + events := make([]*xatu.DecoratedEvent, batchSize) + for i := range events { + events[i] = event + } b.ReportAllocs() b.ResetTimer() - for i := 0; i < b.N; i++ { - w.Write(benchTable, event) - } - - b.StopTimer() - - ctx, ctxCancel := context.WithTimeout(context.Background(), 10*time.Second) - defer ctxCancel() + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + tableEvents := map[string][]*xatu.DecoratedEvent{ + benchTable: events, + } - if err := w.FlushTables(ctx, []string{benchTable}); err != nil { - b.Logf("FlushTables: %v", err) - } + if err := w.FlushTableEvents(context.Background(), tableEvents); err != nil { + b.Logf("FlushTableEvents: %v", err) + } + } + }) - // Count total flushes by reading the metrics. - b.ReportMetric(float64(b.N)/b.Elapsed().Seconds(), "events/sec") + b.StopTimer() + b.ReportMetric(float64(b.N*batchSize)/b.Elapsed().Seconds(), "events/sec") } // BenchmarkEndToEndWithFlatten measures the full cost of processing one -// event from proto through FlattenTo and the Write path to batch-ready. +// event from proto through FlattenTo to batch-ready INSERT. func BenchmarkEndToEndWithFlatten(b *testing.B) { - const bufferSize = 50_000 - var flushCount atomic.Int64 ns := fmt.Sprintf("xatu_bench_e2e_%d", time.Now().UnixNano()) @@ -237,12 +209,7 @@ func BenchmarkEndToEndWithFlatten(b *testing.B) { w := &ChGoWriter{ log: log.WithField("component", "bench_e2e"), metrics: metrics, - config: &Config{ - DrainTimeout: 30 * time.Second, - Defaults: TableConfig{ - BufferSize: bufferSize, - }, - }, + config: &Config{}, chgoCfg: ChGoConfig{ MaxRetries: 0, RetryBaseDelay: 100 * time.Millisecond, @@ -250,7 +217,6 @@ func BenchmarkEndToEndWithFlatten(b *testing.B) { }, tables: make(map[string]*chTableWriter, 4), batchFactories: make(map[string]func() route.ColumnarBatch, 4), - done: make(chan struct{}), poolDoFn: func(_ context.Context, _ ch.Query) error { flushCount.Add(1) @@ -263,78 +229,25 @@ func BenchmarkEndToEndWithFlatten(b *testing.B) { w.RegisterBatchFactories(routes) - var wg sync.WaitGroup - event := benchHeadEvent() b.ReportAllocs() b.ResetTimer() - wg.Add(1) - - go func() { - defer wg.Done() - - for i := 0; i < b.N; i++ { - w.Write(benchTable, event) + for i := 0; i < b.N; i++ { + tableEvents := map[string][]*xatu.DecoratedEvent{ + benchTable: {event}, } - }() - - wg.Wait() - - b.StopTimer() - ctx, ctxCancel := context.WithTimeout(context.Background(), 10*time.Second) - defer ctxCancel() - - if err := w.FlushTables(ctx, []string{benchTable}); err != nil { - b.Logf("FlushTables: %v", err) + if err := w.FlushTableEvents(context.Background(), tableEvents); err != nil { + b.Fatalf("FlushTableEvents: %v", err) + } } - // Signal shutdown. - w.stopOnce.Do(func() { - close(w.done) - }) + b.StopTimer() elapsed := b.Elapsed().Seconds() b.ReportMetric(float64(b.N)/elapsed, "events/sec") b.ReportMetric(float64(flushCount.Load()), "flushes") b.ReportMetric(float64(b.N)/float64(max(flushCount.Load(), 1)), "events/flush") } - -// BenchmarkConcurrentFlush measures throughput when multiple goroutines -// write and flush concurrently, exercising the concurrent INSERT path. -func BenchmarkConcurrentFlush(b *testing.B) { - const ( - bufferSize = 100_000 - flushers = 8 - batchSize = 500 - ) - - w, cancel := benchWriter(b, bufferSize) - defer cancel() - - event := benchHeadEvent() - - // Pre-create the table writer. - w.Write(benchTable, event) - - b.ReportAllocs() - b.ResetTimer() - - b.RunParallel(func(pb *testing.PB) { - for pb.Next() { - // Write a small batch then flush, simulating concurrent WriteBatch calls. - for j := 0; j < batchSize; j++ { - w.Write(benchTable, event) - } - - if err := w.FlushTables(context.Background(), []string{benchTable}); err != nil { - b.Logf("FlushTables: %v", err) - } - } - }) - - b.StopTimer() - b.ReportMetric(float64(b.N*batchSize)/b.Elapsed().Seconds(), "events/sec") -} diff --git a/pkg/consumoor/clickhouse/writer_test.go b/pkg/consumoor/clickhouse/writer_test.go index af6cc6d0..7ec0626f 100644 --- a/pkg/consumoor/clickhouse/writer_test.go +++ b/pkg/consumoor/clickhouse/writer_test.go @@ -42,14 +42,8 @@ func newTestWriter(maxRetries int, baseDelay, maxDelay time.Duration) *ChGoWrite log.SetLevel(logrus.DebugLevel) return &ChGoWriter{ - log: log.WithField("component", "test"), - config: &Config{ - DSN: "clickhouse://localhost:9000/default", - DrainTimeout: 30 * time.Second, - Defaults: TableConfig{ - BufferSize: 100, - }, - }, + log: log.WithField("component", "test"), + config: &Config{}, metrics: sharedTestMetrics(), chgoCfg: ChGoConfig{ MaxRetries: maxRetries, @@ -59,7 +53,6 @@ func newTestWriter(maxRetries int, baseDelay, maxDelay time.Duration) *ChGoWrite }, tables: make(map[string]*chTableWriter, 16), batchFactories: make(map[string]func() route.ColumnarBatch, 8), - done: make(chan struct{}), } } @@ -430,29 +423,26 @@ func TestGetOrCreateTableWriter_DifferentTablesGetDifferentWriters(t *testing.T) assert.Len(t, w.tables, 2) } -func TestFlushTables_ConcurrentDrain(t *testing.T) { +func TestFlushTableEvents_ConcurrentFlush(t *testing.T) { w := newTestWriter(0, time.Millisecond, time.Millisecond) + + var flushCount atomic.Int32 + w.poolDoFn = func(_ context.Context, _ ch.Query) error { + flushCount.Add(1) + return nil } const table = "beacon_block" - // Register a batch factory so flush actually works. w.batchFactories[table] = func() route.ColumnarBatch { return &stubBatch{} } - tw := w.getOrCreateTableWriter(table) - - // Fill the buffer with some events. - const eventCount = 50 - - for i := 0; i < eventCount; i++ { - tw.buffer <- eventEntry{} - } + event := &xatu.DecoratedEvent{} - // Launch multiple concurrent FlushTables calls. + // Launch multiple concurrent FlushTableEvents calls. const flushers = 10 errs := make([]error, flushers) @@ -465,7 +455,9 @@ func TestFlushTables_ConcurrentDrain(t *testing.T) { go func(idx int) { defer wg.Done() - errs[idx] = w.FlushTables(context.Background(), []string{table}) + errs[idx] = w.FlushTableEvents(context.Background(), map[string][]*xatu.DecoratedEvent{ + table: {event}, + }) }(i) } @@ -475,42 +467,7 @@ func TestFlushTables_ConcurrentDrain(t *testing.T) { assert.NoError(t, err, "flusher %d returned error", i) } - // Buffer should be fully drained. - assert.Equal(t, 0, len(tw.buffer)) -} - -func TestStop_DrainsRemainingEvents(t *testing.T) { - w := newTestWriter(0, time.Millisecond, time.Millisecond) - - var flushed atomic.Int32 - - w.poolDoFn = func(_ context.Context, _ ch.Query) error { - flushed.Add(1) - - return nil - } - - const table = "beacon_block" - - w.batchFactories[table] = func() route.ColumnarBatch { - return &stubBatch{} - } - - tw := w.getOrCreateTableWriter(table) - - // Buffer some events. - const eventCount = 25 - - for i := 0; i < eventCount; i++ { - tw.buffer <- eventEntry{} - } - - // Stop should drain and flush remaining events. - err := w.Stop(context.Background()) - require.NoError(t, err) - - assert.Greater(t, flushed.Load(), int32(0), "should have flushed at least once") - assert.Equal(t, 0, len(tw.buffer), "buffer should be empty after stop") + assert.Equal(t, int32(flushers), flushCount.Load(), "each flush should have triggered a pool.Do call") } // stubBatch is a minimal ColumnarBatch implementation for unit tests. @@ -535,7 +492,6 @@ func TestTableConfigMergeSkipFlattenErrors(t *testing.T) { cfg := &Config{ DSN: "clickhouse://localhost", Defaults: TableConfig{ - BufferSize: 1000, SkipFlattenErrors: true, }, } @@ -546,9 +502,6 @@ func TestTableConfigMergeSkipFlattenErrors(t *testing.T) { t.Run("override enables skip", func(t *testing.T) { cfg := &Config{ DSN: "clickhouse://localhost", - Defaults: TableConfig{ - BufferSize: 1000, - }, Tables: map[string]TableConfig{ "some_table": {SkipFlattenErrors: true}, }, @@ -561,7 +514,6 @@ func TestTableConfigMergeSkipFlattenErrors(t *testing.T) { cfg := &Config{ DSN: "clickhouse://localhost", Defaults: TableConfig{ - BufferSize: 1000, SkipFlattenErrors: true, }, Tables: map[string]TableConfig{ @@ -575,9 +527,6 @@ func TestTableConfigMergeSkipFlattenErrors(t *testing.T) { t.Run("both false stays false", func(t *testing.T) { cfg := &Config{ DSN: "clickhouse://localhost", - Defaults: TableConfig{ - BufferSize: 1000, - }, Tables: map[string]TableConfig{ "some_table": {}, }, diff --git a/pkg/consumoor/config_test.go b/pkg/consumoor/config_test.go index 66b664bd..215cacf0 100644 --- a/pkg/consumoor/config_test.go +++ b/pkg/consumoor/config_test.go @@ -32,16 +32,8 @@ func validChGoConfig() clickhouse.ChGoConfig { // validClickHouseConfig returns a clickhouse.Config that passes validation. func validClickHouseConfig() *clickhouse.Config { return &clickhouse.Config{ - DSN: "clickhouse://localhost:9000/default", - Defaults: clickhouse.TableConfig{ - BatchSize: 1000, - FlushInterval: time.Second, - BufferSize: 1000, - }, - OrganicRetryInitDelay: time.Second, - OrganicRetryMaxDelay: 30 * time.Second, - DrainTimeout: 30 * time.Second, - ChGo: validChGoConfig(), + DSN: "clickhouse://localhost:9000/default", + ChGo: validChGoConfig(), } } @@ -56,6 +48,7 @@ func validKafkaConfig() *source.KafkaConfig { SessionTimeoutMs: 30000, CommitInterval: 5 * time.Second, ShutdownTimeout: 30 * time.Second, + MaxInFlight: 64, } } @@ -134,9 +127,6 @@ func TestClickHouseConfigValidateChGo(t *testing.T) { func TestTableConfigForMergesInsertSettings(t *testing.T) { cfg := &clickhouse.Config{ Defaults: clickhouse.TableConfig{ - BatchSize: 100, - FlushInterval: time.Second, - BufferSize: 300, InsertSettings: map[string]any{ "insert_quorum": 2, "insert_quorum_timeout": 30000, @@ -144,7 +134,6 @@ func TestTableConfigForMergesInsertSettings(t *testing.T) { }, Tables: map[string]clickhouse.TableConfig{ "canonical_beacon_block": { - BatchSize: 500, InsertSettings: map[string]any{ "insert_quorum": 3, }, @@ -154,8 +143,6 @@ func TestTableConfigForMergesInsertSettings(t *testing.T) { got := cfg.TableConfigFor("canonical_beacon_block") - assert.Equal(t, 500, got.BatchSize) - assert.Equal(t, 300, got.BufferSize) assert.Equal( t, map[string]any{ @@ -247,47 +234,16 @@ func TestKafkaConfigValidateSessionTimeout(t *testing.T) { }) } -func TestClickHouseConfigValidateBufferSize(t *testing.T) { - t.Run("rejects bufferSize less than batchSize", func(t *testing.T) { - cfg := validClickHouseConfig() - cfg.Defaults.BatchSize = 1000 - cfg.Defaults.BufferSize = 500 - - err := cfg.Validate() - require.Error(t, err) - assert.Contains(t, err.Error(), "bufferSize must be >= batchSize") - }) - - t.Run("accepts bufferSize equal to batchSize", func(t *testing.T) { - cfg := validClickHouseConfig() - cfg.Defaults.BatchSize = 1000 - cfg.Defaults.BufferSize = 1000 - require.NoError(t, cfg.Validate()) - }) -} - func TestTableConfigForAppliesCanonicalDefaults(t *testing.T) { t.Run("adds auto quorum for canonical tables when unset", func(t *testing.T) { - cfg := &clickhouse.Config{ - Defaults: clickhouse.TableConfig{ - BatchSize: 100, - FlushInterval: time.Second, - BufferSize: 300, - }, - } + cfg := &clickhouse.Config{} got := cfg.TableConfigFor("canonical_beacon_block") assert.Equal(t, map[string]any{"insert_quorum": "auto"}, got.InsertSettings) }) t.Run("does not add quorum for non-canonical tables", func(t *testing.T) { - cfg := &clickhouse.Config{ - Defaults: clickhouse.TableConfig{ - BatchSize: 100, - FlushInterval: time.Second, - BufferSize: 300, - }, - } + cfg := &clickhouse.Config{} got := cfg.TableConfigFor("beacon_api_eth_v1_events_head") assert.Nil(t, got.InsertSettings) @@ -295,11 +251,6 @@ func TestTableConfigForAppliesCanonicalDefaults(t *testing.T) { t.Run("preserves explicit per-table quorum", func(t *testing.T) { cfg := &clickhouse.Config{ - Defaults: clickhouse.TableConfig{ - BatchSize: 100, - FlushInterval: time.Second, - BufferSize: 300, - }, Tables: map[string]clickhouse.TableConfig{ "canonical_beacon_block": { InsertSettings: map[string]any{ @@ -316,9 +267,6 @@ func TestTableConfigForAppliesCanonicalDefaults(t *testing.T) { t.Run("preserves explicit default quorum", func(t *testing.T) { cfg := &clickhouse.Config{ Defaults: clickhouse.TableConfig{ - BatchSize: 100, - FlushInterval: time.Second, - BufferSize: 300, InsertSettings: map[string]any{ "insert_quorum": 2, }, diff --git a/pkg/consumoor/health_test.go b/pkg/consumoor/health_test.go index 28f0fcc6..c06c3ca9 100644 --- a/pkg/consumoor/health_test.go +++ b/pkg/consumoor/health_test.go @@ -19,12 +19,12 @@ type mockWriter struct { pingErr error } -func (m *mockWriter) Start(_ context.Context) error { return nil } -func (m *mockWriter) Stop(_ context.Context) error { return nil } -func (m *mockWriter) Write(_ string, _ *xatu.DecoratedEvent) {} -func (m *mockWriter) FlushAll(_ context.Context) error { return nil } -func (m *mockWriter) FlushTables(_ context.Context, _ []string) error { return nil } -func (m *mockWriter) Ping(_ context.Context) error { return m.pingErr } +func (m *mockWriter) Start(_ context.Context) error { return nil } +func (m *mockWriter) Stop(_ context.Context) error { return nil } +func (m *mockWriter) FlushTableEvents(_ context.Context, _ map[string][]*xatu.DecoratedEvent) error { + return nil +} +func (m *mockWriter) Ping(_ context.Context) error { return m.pingErr } func TestHandleHealthz(t *testing.T) { c := &Consumoor{ diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go index e6f5c55d..25c9b9a2 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go @@ -112,10 +112,11 @@ func (b *beaconApiEthV1BeaconBlobBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go index f7d830e3..6d0deed4 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go @@ -106,10 +106,11 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) appendMetadata(event *xatu.Decorate b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.go index d340dd28..6b9a3f15 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.go @@ -2,6 +2,7 @@ package beacon import ( "fmt" + "strconv" "strings" "time" @@ -92,7 +93,7 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) appendPayload(event *xatu.Decorated } if committeeIndex := committee.GetIndex(); committeeIndex != nil { - b.CommitteeIndex.Append(fmt.Sprint(committeeIndex.GetValue())) + b.CommitteeIndex.Append(strconv.FormatUint(committeeIndex.GetValue(), 10)) } else { b.CommitteeIndex.Append("") } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go index 27dc88d1..5e491abf 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go @@ -118,10 +118,11 @@ func (b *beaconApiEthV1EventsAttestationBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.go index d29a35ad..c2a05559 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.go @@ -2,6 +2,7 @@ package beacon import ( "fmt" + "strconv" "time" "github.com/ClickHouse/ch-go/proto" @@ -111,7 +112,7 @@ func (b *beaconApiEthV1EventsAttestationBatch) appendPayload(event *xatu.Decorat } if committeeIndex := data.GetIndex(); committeeIndex != nil { - b.CommitteeIndex.Append(fmt.Sprint(committeeIndex.GetValue())) + b.CommitteeIndex.Append(strconv.FormatUint(committeeIndex.GetValue(), 10)) } else { b.CommitteeIndex.Append("") } @@ -185,7 +186,7 @@ func (b *beaconApiEthV1EventsAttestationBatch) appendAdditionalData(event *xatu. } if ci := av.GetCommitteeIndex(); ci != nil { - b.AttestingValidatorCommitteeIndex.Append(fmt.Sprint(ci.GetValue())) + b.AttestingValidatorCommitteeIndex.Append(strconv.FormatUint(ci.GetValue(), 10)) } else { b.AttestingValidatorCommitteeIndex.Append("") } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go index a9a5c43d..742c1bc2 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go @@ -111,10 +111,11 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go index 3eb8d9ae..5c4c30af 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go @@ -107,10 +107,11 @@ func (b *beaconApiEthV1EventsBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go index c1b75abd..1217a15e 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go @@ -106,10 +106,11 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go index 27e27c16..68cc31dc 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go @@ -114,10 +114,11 @@ func (b *beaconApiEthV1EventsChainReorgBatch) appendMetadata(event *xatu.Decorat b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go index 2fdb44ce..78ee3af5 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go @@ -112,10 +112,11 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) appendMetadata(event *xa b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.go index 542afa65..0323241e 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.go @@ -2,6 +2,7 @@ package beacon import ( "fmt" + "strconv" "time" "github.com/ethpandaops/xatu/pkg/consumoor/route" @@ -131,7 +132,7 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) appendPayload(event *xat b.ContributionBeaconBlockRoot.Append([]byte(contribution.GetBeaconBlockRoot())) if subcommitteeIndex := contribution.GetSubcommitteeIndex(); subcommitteeIndex != nil { - b.ContributionSubcommitteeIndex.Append(fmt.Sprint(subcommitteeIndex.GetValue())) + b.ContributionSubcommitteeIndex.Append(strconv.FormatUint(subcommitteeIndex.GetValue(), 10)) } else { b.ContributionSubcommitteeIndex.Append("") } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go index 66d0b46d..dd1e3412 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go @@ -108,10 +108,11 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendMetadata(event *xatu. b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go index b4d8a790..13d428d0 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go @@ -106,10 +106,11 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) appendMetadata(event *xat b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go index 5ded9078..2f5357d8 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go @@ -112,10 +112,11 @@ func (b *beaconApiEthV1EventsHeadBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go index 5df6cd65..c924dda4 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go @@ -107,10 +107,11 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) appendMetadata(event *xatu.Deco b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go index 6efe2264..8f3f2be4 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go @@ -105,10 +105,11 @@ func (b *beaconApiEthV1ProposerDutyBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go index 30c9d6c5..b8ddf4db 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go @@ -117,10 +117,11 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) appendMetadata(event *xatu b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.go index ffe1e39e..0a925ad2 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.go @@ -2,6 +2,7 @@ package beacon import ( "fmt" + "strconv" "time" "github.com/ethpandaops/xatu/pkg/consumoor/route" @@ -96,7 +97,7 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) appendPayload(event *xatu. } if committeeIndex := attestationData.GetIndex(); committeeIndex != nil { - b.CommitteeIndex.Append(fmt.Sprint(committeeIndex.GetValue())) + b.CommitteeIndex.Append(strconv.FormatUint(committeeIndex.GetValue(), 10)) } else { b.CommitteeIndex.Append("") } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go index 14e8ffed..02b4250b 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go @@ -143,10 +143,11 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go index 47240e04..48749a6f 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go @@ -129,10 +129,11 @@ func (b *beaconApiEthV3ValidatorBlockBatch) appendMetadata(event *xatu.Decorated b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go index b69b1a61..5cf5a719 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go @@ -116,10 +116,11 @@ func (b *canonicalBeaconBlobSidecarBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go index 421b8281..f4c47105 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go @@ -143,10 +143,11 @@ func (b *canonicalBeaconBlockBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go index 3b07a949..4919c8a0 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go @@ -130,10 +130,11 @@ func (b *canonicalBeaconBlockAttesterSlashingBatch) appendMetadata(event *xatu.D b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go index d6beb638..f9edb776 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go @@ -109,10 +109,11 @@ func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) appendMetadata(event *xa b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go index ec7c96d5..043fe41c 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go @@ -111,10 +111,11 @@ func (b *canonicalBeaconBlockDepositBatch) appendMetadata(event *xatu.DecoratedE b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.go b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.go index 557f120c..d36a5d34 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.go @@ -2,6 +2,7 @@ package canonical import ( "fmt" + "strconv" "time" "github.com/ethpandaops/xatu/pkg/consumoor/route" @@ -65,7 +66,7 @@ func (b *canonicalBeaconBlockDepositBatch) appendPayload(event *xatu.DecoratedEv b.DepositDataSignature.Append(data.GetSignature()) if amount := data.GetAmount(); amount != nil { - parsedAmount, err := route.ParseUInt128(fmt.Sprintf("%d", amount.GetValue())) + parsedAmount, err := route.ParseUInt128(strconv.FormatUint(amount.GetValue(), 10)) if err != nil { return fmt.Errorf("parsing deposit_data_amount: %w", err) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go index 106e60ce..8931c837 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go @@ -132,10 +132,11 @@ func (b *canonicalBeaconBlockExecutionTransactionBatch) appendMetadata(event *xa b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go index ac985c01..9c188622 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go @@ -122,10 +122,11 @@ func (b *canonicalBeaconBlockProposerSlashingBatch) appendMetadata(event *xatu.D b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go index 17890132..d694eebd 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go @@ -114,10 +114,11 @@ func (b *canonicalBeaconBlockSyncAggregateBatch) appendMetadata(event *xatu.Deco b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go index a3e92396..563c937c 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go @@ -107,10 +107,11 @@ func (b *canonicalBeaconBlockVoluntaryExitBatch) appendMetadata(event *xatu.Deco b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go index 708ff352..13424507 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go @@ -109,10 +109,11 @@ func (b *canonicalBeaconBlockWithdrawalBatch) appendMetadata(event *xatu.Decorat b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.go b/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.go index a390ccf4..4558630e 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.go @@ -2,6 +2,7 @@ package canonical import ( "fmt" + "strconv" "time" "github.com/ethpandaops/xatu/pkg/consumoor/route" @@ -95,7 +96,7 @@ func (b *canonicalBeaconBlockWithdrawalBatch) appendPayload(event *xatu.Decorate } if amount := withdrawal.GetAmount(); amount != nil { - parsedAmount, err := route.ParseUInt128(fmt.Sprintf("%d", amount.GetValue())) + parsedAmount, err := route.ParseUInt128(strconv.FormatUint(amount.GetValue(), 10)) if err != nil { return fmt.Errorf("parsing withdrawal_amount: %w", err) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go index 4033d32e..b3651dee 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go @@ -104,10 +104,11 @@ func (b *canonicalBeaconCommitteeBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_committee.go b/pkg/consumoor/route/canonical/canonical_beacon_committee.go index f7510e8c..379d78a1 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_committee.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_committee.go @@ -2,6 +2,7 @@ package canonical import ( "fmt" + "strconv" "strings" "time" @@ -66,7 +67,7 @@ func (b *canonicalBeaconCommitteeBatch) appendRuntime(_ *xatu.DecoratedEvent) { func (b *canonicalBeaconCommitteeBatch) appendPayload(event *xatu.DecoratedEvent) { committee := event.GetEthV1BeaconCommittee() if index := committee.GetIndex(); index != nil { - b.CommitteeIndex.Append(fmt.Sprintf("%d", index.GetValue())) + b.CommitteeIndex.Append(strconv.FormatUint(index.GetValue(), 10)) } else { b.CommitteeIndex.Append("") } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go index 3a70daf1..44986dad 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go @@ -121,10 +121,11 @@ func (b *canonicalBeaconElaboratedAttestationBatch) appendMetadata(event *xatu.D b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.go b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.go index 738af99d..213a6df9 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.go @@ -1,7 +1,7 @@ package canonical import ( - "fmt" + "strconv" "time" "github.com/ethpandaops/xatu/pkg/consumoor/route" @@ -61,7 +61,7 @@ func (b *canonicalBeaconElaboratedAttestationBatch) appendRow(event *xatu.Decora b.BeaconBlockRoot.Append([]byte(data.GetBeaconBlockRoot())) if index := data.GetIndex(); index != nil { - b.CommitteeIndex.Append(fmt.Sprintf("%d", index.GetValue())) + b.CommitteeIndex.Append(strconv.FormatUint(index.GetValue(), 10)) } else { b.CommitteeIndex.Append("") } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go index 78347998..ac2c8510 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go @@ -103,10 +103,11 @@ func (b *canonicalBeaconProposerDutyBatch) appendMetadata(event *xatu.DecoratedE b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go index 12023f4a..80d420db 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go @@ -104,10 +104,11 @@ func (b *canonicalBeaconSyncCommitteeBatch) appendMetadata(event *xatu.Decorated b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go index 406f9d92..7765b16c 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go @@ -114,10 +114,11 @@ func (b *canonicalBeaconValidatorsBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go index 0fad0203..2170bf85 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go @@ -102,10 +102,11 @@ func (b *canonicalBeaconValidatorsPubkeysBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go index 49b12616..6943e268 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go @@ -102,10 +102,11 @@ func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) appendMetadata(eve b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - b.MetaConsensusVersion.Append(route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMajor.Append(route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionMinor.Append(route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) - b.MetaConsensusVersionPatch.Append(route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())) + cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) + b.MetaConsensusVersion.Append(cvNorm) + b.MetaConsensusVersionMajor.Append(cvMajor) + b.MetaConsensusVersionMinor.Append(cvMinor) + b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { b.MetaLabels.Append(labels) diff --git a/pkg/consumoor/route/cmd/chgo-rowgen/main.go b/pkg/consumoor/route/cmd/chgo-rowgen/main.go index 52ae1aa7..9d32a23c 100644 --- a/pkg/consumoor/route/cmd/chgo-rowgen/main.go +++ b/pkg/consumoor/route/cmd/chgo-rowgen/main.go @@ -1024,10 +1024,10 @@ var metaColumnDefs = map[string]metaColumnDef{ // Consensus client "meta_consensus_implementation": {expr: "event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()"}, - "meta_consensus_version": {expr: "route.NormalizeConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())"}, - "meta_consensus_version_major": {expr: "route.ConsensusVersionMajor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())"}, - "meta_consensus_version_minor": {expr: "route.ConsensusVersionMinor(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())"}, - "meta_consensus_version_patch": {expr: "route.ConsensusVersionPatch(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())"}, + "meta_consensus_version": {special: "consensus_version_normalized"}, + "meta_consensus_version_major": {special: "consensus_version_major"}, + "meta_consensus_version_minor": {special: "consensus_version_minor"}, + "meta_consensus_version_patch": {special: "consensus_version_patch"}, // Execution client "meta_execution_implementation": {expr: "event.GetMeta().GetClient().GetEthereum().GetExecution().GetImplementation()"}, @@ -1059,6 +1059,26 @@ func writeAppendMetadata(b *bytes.Buffer, batchName string, cols []column, gens b.WriteString("\t}\n\n") // Non-nil case: append from proto getters. + // + // Emit a single ParseConsensusVersion call if any of the four + // consensus_version columns are present, so the version string is + // parsed once instead of four separate times per event. + cvEmitted := false + cvFieldMap := map[string]string{ + "consensus_version_normalized": "", + "consensus_version_major": "", + "consensus_version_minor": "", + "consensus_version_patch": "", + } + + for _, col := range cols { + if def, ok := metaColumnDefs[col.Name]; ok { + if _, isCv := cvFieldMap[def.special]; isCv { + cvFieldMap[def.special] = col.Field + } + } + } + for i, col := range cols { if !strings.HasPrefix(col.Name, "meta_") { continue @@ -1085,6 +1105,33 @@ func writeAppendMetadata(b *bytes.Buffer, batchName string, cols []column, gens continue } + // Consensus version columns: emit the single parse call on + // first encounter, then reference the local variables. + if _, isCv := cvFieldMap[def.special]; isCv { + if !cvEmitted { + b.WriteString("\tcvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion())\n") + + cvEmitted = true + } + + var localVar string + + switch def.special { + case "consensus_version_normalized": + localVar = "cvNorm" + case "consensus_version_major": + localVar = "cvMajor" + case "consensus_version_minor": + localVar = "cvMinor" + case "consensus_version_patch": + localVar = "cvPatch" + } + + fmt.Fprintf(b, "\tb.%s.Append(%s)\n", col.Field, localVar) + + continue + } + expr := metaAppendExpr(&gen, def) fmt.Fprintf(b, "\tb.%s.Append(%s)\n", col.Field, expr) } diff --git a/pkg/consumoor/route/column.go b/pkg/consumoor/route/column.go index 5312a7f6..bd17c2f5 100644 --- a/pkg/consumoor/route/column.go +++ b/pkg/consumoor/route/column.go @@ -71,12 +71,13 @@ func ConsensusVersionPatch(raw string) string { return patch } -// parseConsensusVersion normalizes and splits a consensus version string -// into major, minor, patch components. -func parseConsensusVersion(raw string) (major, minor, patch string) { - normalized := NormalizeConsensusVersion(raw) +// ParseConsensusVersion normalizes a raw consensus version string and +// splits it into its four components in a single pass. +// This avoids repeated SplitN calls when each component is needed. +func ParseConsensusVersion(raw string) (normalized, major, minor, patch string) { + normalized = NormalizeConsensusVersion(raw) if normalized == "" { - return "", "", "" + return "", "", "", "" } version := strings.TrimPrefix(normalized, "v") @@ -99,6 +100,14 @@ func parseConsensusVersion(raw string) (major, minor, patch string) { } } + return normalized, major, minor, patch +} + +// parseConsensusVersion normalizes and splits a consensus version string +// into major, minor, patch components. +func parseConsensusVersion(raw string) (major, minor, patch string) { + _, major, minor, patch = ParseConsensusVersion(raw) + return major, minor, patch } diff --git a/pkg/consumoor/source/benthos.go b/pkg/consumoor/source/benthos.go index f18f7921..453ca064 100644 --- a/pkg/consumoor/source/benthos.go +++ b/pkg/consumoor/source/benthos.go @@ -69,7 +69,7 @@ func NewBenthosStream( metrics: metrics, rejectSink: rejectSink, ownsWriter: ownsWriter, - }, batchPolicy, 1, nil + }, batchPolicy, kafkaConfig.MaxInFlight, nil }, ); registerErr != nil { closeRejectSink() diff --git a/pkg/consumoor/source/kafka.go b/pkg/consumoor/source/kafka.go index 13bba05d..34f90aa1 100644 --- a/pkg/consumoor/source/kafka.go +++ b/pkg/consumoor/source/kafka.go @@ -13,6 +13,8 @@ import ( const unknownKafkaTopic = "unknown" +var jsonUnmarshalOpts = protojson.UnmarshalOptions{DiscardUnknown: true} + type kafkaMessageMetadata struct { Topic string Partition int32 @@ -101,8 +103,7 @@ func decodeDecoratedEvent(encoding string, data []byte) (*xatu.DecoratedEvent, e return nil, fmt.Errorf("protobuf unmarshal: %w", err) } default: - opts := protojson.UnmarshalOptions{DiscardUnknown: true} - if err := opts.Unmarshal(data, event); err != nil { + if err := jsonUnmarshalOpts.Unmarshal(data, event); err != nil { return nil, fmt.Errorf("json unmarshal: %w", err) } } diff --git a/pkg/consumoor/source/lag_monitor.go b/pkg/consumoor/source/lag_monitor.go index 5a2fe0d5..75b14ff1 100644 --- a/pkg/consumoor/source/lag_monitor.go +++ b/pkg/consumoor/source/lag_monitor.go @@ -30,8 +30,8 @@ type LagMonitor struct { } // NewLagMonitor creates a new LagMonitor. Call Start to begin polling. -// The consumerGroups slice should contain the actual per-topic consumer -// group names used by the Benthos streams (e.g. "base-group-topicA"). +// The consumerGroups slice contains the consumer group names to monitor +// for lag (typically the single base consumer group shared by all streams). func NewLagMonitor( log logrus.FieldLogger, cfg *KafkaConfig, @@ -119,7 +119,7 @@ func (m *LagMonitor) Stop() error { } // poll uses kadm.Client.Lag to fetch and publish consumer group lag -// for all per-topic consumer groups. +// for all monitored consumer groups. func (m *LagMonitor) poll(ctx context.Context) { lags, err := m.admClient.Lag(ctx, m.consumerGroups...) if err != nil { diff --git a/pkg/consumoor/source/output.go b/pkg/consumoor/source/output.go index 650963ca..9c97f311 100644 --- a/pkg/consumoor/source/output.go +++ b/pkg/consumoor/source/output.go @@ -174,7 +174,7 @@ func (o *xatuClickHouseOutput) WriteBatch( g.messages = append(g.messages, groupMessage{ batchIndex: i, - raw: append([]byte(nil), raw...), + raw: raw, event: event, kafka: kafka, tables: tables, @@ -182,10 +182,11 @@ func (o *xatuClickHouseOutput) WriteBatch( } // Phase 2: process each event group independently. + // Pass batchErr through so Phase 1 failures (decode errors) are preserved + // when a group also fails — otherwise processGroup would create a new + // BatchError that silently drops the earlier failures. for _, g := range groups { - if err := o.processGroup(ctx, msgs, g); err != nil { - batchErr = err - } + batchErr = o.processGroup(ctx, msgs, batchErr, g) } if batchErr != nil { @@ -226,43 +227,35 @@ func (o *xatuClickHouseOutput) Close(ctx context.Context) error { // processGroup writes all messages in the group to their target tables, then // flushes only those tables. On failure the entire group is NAK'd or DLQ'd. +// The caller's accumulated batchErr is threaded through so that failures from +// earlier phases (e.g. decode errors) are preserved. func (o *xatuClickHouseOutput) processGroup( ctx context.Context, msgs service.MessageBatch, + batchErr *service.BatchError, g *eventGroup, ) *service.BatchError { - // Collect the unique set of tables for this group. - tableSet := make(map[string]struct{}, 4) - + tableEvents := make(map[string][]*xatu.DecoratedEvent, 4) for _, gm := range g.messages { for _, table := range gm.tables { - tableSet[table] = struct{}{} - } - - for _, table := range gm.tables { - o.writer.Write(table, gm.event) + tableEvents[table] = append(tableEvents[table], gm.event) } } - tables := make([]string, 0, len(tableSet)) - for t := range tableSet { - tables = append(tables, t) - } - - err := o.writer.FlushTables(ctx, tables) + err := o.writer.FlushTableEvents(ctx, tableEvents) if err == nil { - return nil + return batchErr } // Flush failed — attribute to all messages in the group. - var batchErr *service.BatchError - if clickhouse.IsPermanentWriteError(err) { for _, gm := range g.messages { + // Copy raw bytes only when needed for DLQ; the success path + // avoids the copy entirely by referencing the Benthos-owned slice. rejectErr := o.rejectMessage(ctx, &rejectedRecord{ Reason: rejectReasonWritePermanent, Err: err.Error(), - Payload: gm.raw, + Payload: append([]byte(nil), gm.raw...), EventName: gm.event.GetEvent().GetName().String(), Kafka: gm.kafka, }) diff --git a/pkg/consumoor/source/writer.go b/pkg/consumoor/source/writer.go index c3c96a4a..c89a0874 100644 --- a/pkg/consumoor/source/writer.go +++ b/pkg/consumoor/source/writer.go @@ -10,11 +10,10 @@ import ( type Writer interface { Start(ctx context.Context) error Stop(ctx context.Context) error - Write(table string, event *xatu.DecoratedEvent) - // FlushTables forces the specified table writers (by base table name) - // to drain their buffers and write to ClickHouse synchronously. - // An empty or nil slice is a no-op that returns nil. - FlushTables(ctx context.Context, tables []string) error + // FlushTableEvents writes the given events directly to their respective + // ClickHouse tables concurrently. The map keys are base table names + // (without suffix). Returns a joined error containing all table failures. + FlushTableEvents(ctx context.Context, tableEvents map[string][]*xatu.DecoratedEvent) error // Ping checks connectivity to the underlying datastore. Ping(ctx context.Context) error } diff --git a/pkg/consumoor/telemetry/metrics.go b/pkg/consumoor/telemetry/metrics.go index ba8a61d5..ec364eb7 100644 --- a/pkg/consumoor/telemetry/metrics.go +++ b/pkg/consumoor/telemetry/metrics.go @@ -7,22 +7,21 @@ import ( // Metrics holds all Prometheus metrics for the consumoor service. type Metrics struct { - messagesConsumed *prometheus.CounterVec - messagesRouted *prometheus.CounterVec - messagesDropped *prometheus.CounterVec - messagesRejected *prometheus.CounterVec - decodeErrors *prometheus.CounterVec - dlqWrites *prometheus.CounterVec - dlqErrors *prometheus.CounterVec - rowsWritten *prometheus.CounterVec - writeErrors *prometheus.CounterVec - writeDuration *prometheus.HistogramVec - batchSize *prometheus.HistogramVec - bufferUsage *prometheus.GaugeVec - bufferUsageTotal prometheus.Gauge - flattenErrors *prometheus.CounterVec - activeTopics prometheus.Gauge - kafkaConsumerLag *prometheus.GaugeVec + messagesConsumed *prometheus.CounterVec + messagesRouted *prometheus.CounterVec + messagesDropped *prometheus.CounterVec + messagesRejected *prometheus.CounterVec + decodeErrors *prometheus.CounterVec + dlqWrites *prometheus.CounterVec + dlqErrors *prometheus.CounterVec + rowsWritten *prometheus.CounterVec + writeErrors *prometheus.CounterVec + writeDuration *prometheus.HistogramVec + batchSize *prometheus.HistogramVec + flattenErrors *prometheus.CounterVec + activeTopics prometheus.Gauge + kafkaConsumerLag *prometheus.GaugeVec + outputMaxInFlight prometheus.Gauge // adaptive limiter metrics (per-table) adaptiveLimiterLimit *prometheus.GaugeVec @@ -130,20 +129,6 @@ func NewMetrics(namespace string) *Metrics { Buckets: []float64{1, 10, 100, 1000, 10000, 50000, 100000, 200000, 500000, 1000000}, }, []string{"table"}), - bufferUsage: promauto.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "buffer_usage", - Help: "Current number of rows buffered per table.", - }, []string{"table"}), - - bufferUsageTotal: promauto.NewGauge(prometheus.GaugeOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "buffer_usage_total", - Help: "Sum of all table buffer usages. Single number for aggregate memory pressure alerting.", - }), - flattenErrors: promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: namespace, Subsystem: subsystem, @@ -165,6 +150,13 @@ func NewMetrics(namespace string) *Metrics { Help: "Kafka consumer group lag per topic and partition.", }, []string{"topic", "partition", "consumer_group"}), + outputMaxInFlight: promauto.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "output_max_in_flight", + Help: "Configured maximum number of concurrent Benthos WriteBatch calls per stream.", + }), + adaptiveLimiterLimit: promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, @@ -269,11 +261,10 @@ func (m *Metrics) RowsWritten() *prometheus.CounterVec { return m.rowsWritt func (m *Metrics) WriteErrors() *prometheus.CounterVec { return m.writeErrors } func (m *Metrics) WriteDuration() *prometheus.HistogramVec { return m.writeDuration } func (m *Metrics) BatchSize() *prometheus.HistogramVec { return m.batchSize } -func (m *Metrics) BufferUsage() *prometheus.GaugeVec { return m.bufferUsage } -func (m *Metrics) BufferUsageTotal() prometheus.Gauge { return m.bufferUsageTotal } func (m *Metrics) FlattenErrors() *prometheus.CounterVec { return m.flattenErrors } func (m *Metrics) ActiveTopics() prometheus.Gauge { return m.activeTopics } func (m *Metrics) KafkaConsumerLag() *prometheus.GaugeVec { return m.kafkaConsumerLag } +func (m *Metrics) OutputMaxInFlight() prometheus.Gauge { return m.outputMaxInFlight } func (m *Metrics) AdaptiveLimiterLimit() *prometheus.GaugeVec { return m.adaptiveLimiterLimit } func (m *Metrics) AdaptiveLimiterInflight() *prometheus.GaugeVec { return m.adaptiveLimiterInflight } From 8fa34c28f9584ba77bf95070eec156964336443b Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 18:58:06 +1000 Subject: [PATCH 09/27] perf(consumoor): use vtprotobuf for reflection-free proto deserialization Add planetscale/vtprotobuf code generation to eliminate reflect.New allocations during protobuf unmarshaling. CPU profiling showed 502M reflect.New objects from proto.Unmarshal driving ~32% GC overhead. vtprotobuf generates type-specific UnmarshalVT methods that populate struct fields directly without reflection, producing identical wire format bytes. The hot path in decodeDecoratedEvent now calls event.UnmarshalVT(data) instead of proto.Unmarshal(data, event). Changes: - Add planetscale-vtprotobuf plugin to buf.gen.yaml - Generate *_vtproto.pb.go files for all 27 proto packages - Switch consumoor protobuf decode path to UnmarshalVT - Promote vtprotobuf from indirect to direct dependency --- buf.gen.yaml | 5 + go.mod | 1 + go.sum | 2 + pkg/consumoor/source/kafka.go | 3 +- pkg/proto/eth/v1/attestation_vtproto.pb.go | 2088 + pkg/proto/eth/v1/beacon_block_vtproto.pb.go | 5787 ++ pkg/proto/eth/v1/blob_vtproto.pb.go | 897 + pkg/proto/eth/v1/checkpoint_vtproto.pb.go | 373 + pkg/proto/eth/v1/committee_vtproto.pb.go | 268 + pkg/proto/eth/v1/duties_vtproto.pb.go | 259 + pkg/proto/eth/v1/events_vtproto.pb.go | 4753 ++ .../eth/v1/execution_engine_vtproto.pb.go | 9580 +++ .../eth/v1/execution_requests_vtproto.pb.go | 1104 + pkg/proto/eth/v1/fork_choice_vtproto.pb.go | 1440 + pkg/proto/eth/v1/sync_committee_vtproto.pb.go | 966 + pkg/proto/eth/v1/validator_vtproto.pb.go | 810 + pkg/proto/eth/v2/beacon_block_vtproto.pb.go | 14689 +++++ pkg/proto/eth/v2/events_vtproto.pb.go | 1381 + pkg/proto/eth/v2/withdrawals_vtproto.pb.go | 979 + pkg/proto/libp2p/eth_vtproto.pb.go | 710 + pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go | 1104 + pkg/proto/libp2p/peer_vtproto.pb.go | 435 + pkg/proto/libp2p/trace_vtproto.pb.go | 11367 ++++ pkg/proto/mevrelay/bids_vtproto.pb.go | 816 + pkg/proto/mevrelay/payloads_vtproto.pb.go | 666 + pkg/proto/mevrelay/relay_vtproto.pb.go | 704 + pkg/proto/noderecord/consensus_vtproto.pb.go | 1076 + pkg/proto/noderecord/execution_vtproto.pb.go | 868 + pkg/proto/xatu/coordinator_vtproto.pb.go | 11069 ++++ pkg/proto/xatu/event_ingester_vtproto.pb.go | 48622 ++++++++++++++++ 30 files changed, 122820 insertions(+), 2 deletions(-) create mode 100644 pkg/proto/eth/v1/attestation_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/beacon_block_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/blob_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/checkpoint_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/committee_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/duties_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/events_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/execution_engine_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/execution_requests_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/fork_choice_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/sync_committee_vtproto.pb.go create mode 100644 pkg/proto/eth/v1/validator_vtproto.pb.go create mode 100644 pkg/proto/eth/v2/beacon_block_vtproto.pb.go create mode 100644 pkg/proto/eth/v2/events_vtproto.pb.go create mode 100644 pkg/proto/eth/v2/withdrawals_vtproto.pb.go create mode 100644 pkg/proto/libp2p/eth_vtproto.pb.go create mode 100644 pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go create mode 100644 pkg/proto/libp2p/peer_vtproto.pb.go create mode 100644 pkg/proto/libp2p/trace_vtproto.pb.go create mode 100644 pkg/proto/mevrelay/bids_vtproto.pb.go create mode 100644 pkg/proto/mevrelay/payloads_vtproto.pb.go create mode 100644 pkg/proto/mevrelay/relay_vtproto.pb.go create mode 100644 pkg/proto/noderecord/consensus_vtproto.pb.go create mode 100644 pkg/proto/noderecord/execution_vtproto.pb.go create mode 100644 pkg/proto/xatu/coordinator_vtproto.pb.go create mode 100644 pkg/proto/xatu/event_ingester_vtproto.pb.go diff --git a/buf.gen.yaml b/buf.gen.yaml index 656df407..a61bfc81 100644 --- a/buf.gen.yaml +++ b/buf.gen.yaml @@ -6,3 +6,8 @@ plugins: - remote: buf.build/grpc/go:v1.5.1 out: . opt: paths=source_relative + - remote: buf.build/community/planetscale-vtprotobuf + out: . + opt: + - paths=source_relative + - features=marshal+unmarshal+size diff --git a/go.mod b/go.mod index 746a694e..eb44716e 100644 --- a/go.mod +++ b/go.mod @@ -245,6 +245,7 @@ require ( github.com/pion/turn/v4 v4.0.2 // indirect github.com/pion/webrtc/v4 v4.1.2 // indirect github.com/pk910/dynamic-ssz v0.0.6 // indirect + github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55 // indirect github.com/prometheus/common v0.67.1 // indirect diff --git a/go.sum b/go.sum index 1f23646a..45ce7e56 100644 --- a/go.sum +++ b/go.sum @@ -655,6 +655,8 @@ github.com/pk910/dynamic-ssz v0.0.6/go.mod h1:b6CrLaB2X7pYA+OSEEbkgXDEcRnjLOZIxZ github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 h1:GFCKgmp0tecUJ0sJuv4pzYCqS9+RGSn52M3FUwPs+uo= +github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10/go.mod h1:t/avpk3KcrXxUnYOhZhMXJlSEyie6gQbtLq5NM3loB8= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= diff --git a/pkg/consumoor/source/kafka.go b/pkg/consumoor/source/kafka.go index 34f90aa1..955d8201 100644 --- a/pkg/consumoor/source/kafka.go +++ b/pkg/consumoor/source/kafka.go @@ -8,7 +8,6 @@ import ( "github.com/ethpandaops/xatu/pkg/proto/xatu" "google.golang.org/protobuf/encoding/protojson" - "google.golang.org/protobuf/proto" ) const unknownKafkaTopic = "unknown" @@ -99,7 +98,7 @@ func decodeDecoratedEvent(encoding string, data []byte) (*xatu.DecoratedEvent, e switch encoding { case "protobuf": - if err := proto.Unmarshal(data, event); err != nil { + if err := event.UnmarshalVT(data); err != nil { return nil, fmt.Errorf("protobuf unmarshal: %w", err) } default: diff --git a/pkg/proto/eth/v1/attestation_vtproto.pb.go b/pkg/proto/eth/v1/attestation_vtproto.pb.go new file mode 100644 index 00000000..fdd2260f --- /dev/null +++ b/pkg/proto/eth/v1/attestation_vtproto.pb.go @@ -0,0 +1,2088 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/attestation.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *Attestation) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Attestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Attestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Data != nil { + size, err := m.Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if len(m.AggregationBits) > 0 { + i -= len(m.AggregationBits) + copy(dAtA[i:], m.AggregationBits) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.AggregationBits))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *AttestationV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AttestationV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AttestationV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Data != nil { + size, err := m.Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if len(m.AggregationBits) > 0 { + i -= len(m.AggregationBits) + copy(dAtA[i:], m.AggregationBits) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.AggregationBits))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *AttestationData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AttestationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AttestationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if len(m.BeaconBlockRoot) > 0 { + i -= len(m.BeaconBlockRoot) + copy(dAtA[i:], m.BeaconBlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BeaconBlockRoot))) + i-- + dAtA[i] = 0x1a + } + if m.Index != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Index)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *AttestationDataV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AttestationDataV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AttestationDataV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if len(m.BeaconBlockRoot) > 0 { + i -= len(m.BeaconBlockRoot) + copy(dAtA[i:], m.BeaconBlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BeaconBlockRoot))) + i-- + dAtA[i] = 0x1a + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *AggregateAttestationAndProof) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AggregateAttestationAndProof) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AggregateAttestationAndProof) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Aggregate != nil { + size, err := m.Aggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.SelectionProof) > 0 { + i -= len(m.SelectionProof) + copy(dAtA[i:], m.SelectionProof) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.SelectionProof))) + i-- + dAtA[i] = 0x12 + } + if m.AggregatorIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.AggregatorIndex)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SignedAggregateAttestationAndProof) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedAggregateAttestationAndProof) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedAggregateAttestationAndProof) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *AggregateAttestationAndProofV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AggregateAttestationAndProofV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AggregateAttestationAndProofV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Aggregate != nil { + size, err := m.Aggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.AggregatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.AggregatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedAggregateAttestationAndProofV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedAggregateAttestationAndProofV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedAggregateAttestationAndProofV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ElaboratedAttestation) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ElaboratedAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ElaboratedAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ValidatorIndexes) > 0 { + for iNdEx := len(m.ValidatorIndexes) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorIndexes[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if m.Data != nil { + size, err := m.Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Attestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.AggregationBits) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Data != nil { + l = m.Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AttestationV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.AggregationBits) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Data != nil { + l = m.Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AttestationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.Index != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Index)) + } + l = len(m.BeaconBlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AttestationDataV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BeaconBlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AggregateAttestationAndProof) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AggregatorIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.AggregatorIndex)) + } + l = len(m.SelectionProof) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Aggregate != nil { + l = m.Aggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedAggregateAttestationAndProof) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AggregateAttestationAndProofV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AggregatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.AggregatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Aggregate != nil { + l = m.Aggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedAggregateAttestationAndProofV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ElaboratedAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Data != nil { + l = m.Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ValidatorIndexes) > 0 { + for _, e := range m.ValidatorIndexes { + l = (*wrapperspb.UInt64Value)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *Attestation) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Attestation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Attestation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregationBits", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AggregationBits = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &AttestationData{} + } + if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AttestationV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AttestationV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AttestationV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregationBits", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AggregationBits = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &AttestationDataV2{} + } + if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AttestationData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AttestationData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AttestationData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + m.Index = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Index |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeaconBlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BeaconBlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &Checkpoint{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &Checkpoint{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AttestationDataV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AttestationDataV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AttestationDataV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeaconBlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BeaconBlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &CheckpointV2{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &CheckpointV2{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AggregateAttestationAndProof) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AggregateAttestationAndProof: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AggregateAttestationAndProof: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregatorIndex", wireType) + } + m.AggregatorIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.AggregatorIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SelectionProof", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SelectionProof = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Aggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Aggregate == nil { + m.Aggregate = &Attestation{} + } + if err := m.Aggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedAggregateAttestationAndProof) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedAggregateAttestationAndProof: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedAggregateAttestationAndProof: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &AggregateAttestationAndProof{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AggregateAttestationAndProofV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AggregateAttestationAndProofV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AggregateAttestationAndProofV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AggregatorIndex == nil { + m.AggregatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.AggregatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Aggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Aggregate == nil { + m.Aggregate = &AttestationV2{} + } + if err := m.Aggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedAggregateAttestationAndProofV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedAggregateAttestationAndProofV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedAggregateAttestationAndProofV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &AggregateAttestationAndProofV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ElaboratedAttestation) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ElaboratedAttestation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ElaboratedAttestation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &AttestationDataV2{} + } + if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndexes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ValidatorIndexes = append(m.ValidatorIndexes, &wrapperspb1.UInt64Value{}) + if err := (*wrapperspb.UInt64Value)(m.ValidatorIndexes[len(m.ValidatorIndexes)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/beacon_block_vtproto.pb.go b/pkg/proto/eth/v1/beacon_block_vtproto.pb.go new file mode 100644 index 00000000..4cbe516b --- /dev/null +++ b/pkg/proto/eth/v1/beacon_block_vtproto.pb.go @@ -0,0 +1,5787 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/beacon_block.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *BeaconBlock) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ProposerIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlock) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBody) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBody) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBody) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ProposerSlashing) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ProposerSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ProposerSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SignedHeader_2 != nil { + size, err := m.SignedHeader_2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.SignedHeader_1 != nil { + size, err := m.SignedHeader_1.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ProposerSlashingV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ProposerSlashingV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ProposerSlashingV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SignedHeader_2 != nil { + size, err := m.SignedHeader_2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.SignedHeader_1 != nil { + size, err := m.SignedHeader_1.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *AttesterSlashing) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AttesterSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AttesterSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Attestation_2 != nil { + size, err := m.Attestation_2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Attestation_1 != nil { + size, err := m.Attestation_1.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *AttesterSlashingV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AttesterSlashingV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AttesterSlashingV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Attestation_2 != nil { + size, err := m.Attestation_2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Attestation_1 != nil { + size, err := m.Attestation_1.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Deposit_Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Deposit_Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Deposit_Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x22 + } + if m.Amount != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Amount)) + i-- + dAtA[i] = 0x18 + } + if len(m.WithdrawalCredentials) > 0 { + i -= len(m.WithdrawalCredentials) + copy(dAtA[i:], m.WithdrawalCredentials) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.WithdrawalCredentials))) + i-- + dAtA[i] = 0x12 + } + if len(m.Pubkey) > 0 { + i -= len(m.Pubkey) + copy(dAtA[i:], m.Pubkey) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Pubkey))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Deposit) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Deposit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Deposit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Data != nil { + size, err := m.Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.Proof) > 0 { + for iNdEx := len(m.Proof) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Proof[iNdEx]) + copy(dAtA[i:], m.Proof[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Proof[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *DepositV2_Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DepositV2_Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DepositV2_Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x22 + } + if m.Amount != nil { + size, err := (*wrapperspb.UInt64Value)(m.Amount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.WithdrawalCredentials) > 0 { + i -= len(m.WithdrawalCredentials) + copy(dAtA[i:], m.WithdrawalCredentials) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.WithdrawalCredentials))) + i-- + dAtA[i] = 0x12 + } + if len(m.Pubkey) > 0 { + i -= len(m.Pubkey) + copy(dAtA[i:], m.Pubkey) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Pubkey))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DepositV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DepositV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DepositV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Data != nil { + size, err := m.Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.Proof) > 0 { + for iNdEx := len(m.Proof) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Proof[iNdEx]) + copy(dAtA[i:], m.Proof[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Proof[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *VoluntaryExit) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ValidatorIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ValidatorIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Epoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Epoch)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *VoluntaryExitV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VoluntaryExitV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VoluntaryExitV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ValidatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.Epoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedVoluntaryExit) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedVoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedVoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedVoluntaryExitV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedVoluntaryExitV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedVoluntaryExitV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Eth1Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Eth1Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Eth1Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x1a + } + if m.DepositCount != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.DepositCount)) + i-- + dAtA[i] = 0x10 + } + if len(m.DepositRoot) > 0 { + i -= len(m.DepositRoot) + copy(dAtA[i:], m.DepositRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.DepositRoot))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockHeader) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockHeader) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockHeader) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BodyRoot) > 0 { + i -= len(m.BodyRoot) + copy(dAtA[i:], m.BodyRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BodyRoot))) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ProposerIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockHeaderV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockHeaderV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockHeaderV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BodyRoot) > 0 { + i -= len(m.BodyRoot) + copy(dAtA[i:], m.BodyRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BodyRoot))) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockHeader) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockHeader) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockHeader) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockHeaderV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockHeaderV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockHeaderV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *IndexedAttestation) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *IndexedAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *IndexedAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x1a + } + if m.Data != nil { + size, err := m.Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.AttestingIndices) > 0 { + var pksize2 int + for _, num := range m.AttestingIndices { + pksize2 += protohelpers.SizeOfVarint(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num := range m.AttestingIndices { + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = protohelpers.EncodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *IndexedAttestationV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *IndexedAttestationV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *IndexedAttestationV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x1a + } + if m.Data != nil { + size, err := m.Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.AttestingIndices) > 0 { + for iNdEx := len(m.AttestingIndices) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.UInt64Value)(m.AttestingIndices[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SyncAggregate) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncAggregate) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SyncAggregate) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.SyncCommitteeSignature) > 0 { + i -= len(m.SyncCommitteeSignature) + copy(dAtA[i:], m.SyncCommitteeSignature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.SyncCommitteeSignature))) + i-- + dAtA[i] = 0x12 + } + if len(m.SyncCommitteeBits) > 0 { + i -= len(m.SyncCommitteeBits) + copy(dAtA[i:], m.SyncCommitteeBits) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.SyncCommitteeBits))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.ProposerIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ProposerIndex)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBody) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ProposerSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SignedHeader_1 != nil { + l = m.SignedHeader_1.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SignedHeader_2 != nil { + l = m.SignedHeader_2.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ProposerSlashingV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SignedHeader_1 != nil { + l = m.SignedHeader_1.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SignedHeader_2 != nil { + l = m.SignedHeader_2.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AttesterSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Attestation_1 != nil { + l = m.Attestation_1.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Attestation_2 != nil { + l = m.Attestation_2.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AttesterSlashingV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Attestation_1 != nil { + l = m.Attestation_1.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Attestation_2 != nil { + l = m.Attestation_2.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Deposit_Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Pubkey) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.WithdrawalCredentials) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Amount != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Amount)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Deposit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Proof) > 0 { + for _, s := range m.Proof { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.Data != nil { + l = m.Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DepositV2_Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Pubkey) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.WithdrawalCredentials) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Amount != nil { + l = (*wrapperspb.UInt64Value)(m.Amount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DepositV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Proof) > 0 { + for _, s := range m.Proof { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.Data != nil { + l = m.Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *VoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Epoch)) + } + if m.ValidatorIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ValidatorIndex)) + } + n += len(m.unknownFields) + return n +} + +func (m *VoluntaryExitV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = (*wrapperspb.UInt64Value)(m.Epoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ValidatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ValidatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedVoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedVoluntaryExitV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Eth1Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.DepositRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.DepositCount != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.DepositCount)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockHeader) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.ProposerIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ProposerIndex)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BodyRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockHeaderV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BodyRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockHeader) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockHeaderV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *IndexedAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.AttestingIndices) > 0 { + l = 0 + for _, e := range m.AttestingIndices { + l += protohelpers.SizeOfVarint(uint64(e)) + } + n += 1 + protohelpers.SizeOfVarint(uint64(l)) + l + } + if m.Data != nil { + l = m.Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *IndexedAttestationV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.AttestingIndices) > 0 { + for _, e := range m.AttestingIndices { + l = (*wrapperspb.UInt64Value)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.Data != nil { + l = m.Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SyncAggregate) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.SyncCommitteeBits) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.SyncCommitteeSignature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlock) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlock: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlock: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + m.ProposerIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProposerIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBody{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBody{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlock) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlock: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlock: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BeaconBlock{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BeaconBlockV2{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBody) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBody: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBody: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ProposerSlashing) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ProposerSlashing: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ProposerSlashing: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SignedHeader_1", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SignedHeader_1 == nil { + m.SignedHeader_1 = &SignedBeaconBlockHeader{} + } + if err := m.SignedHeader_1.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SignedHeader_2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SignedHeader_2 == nil { + m.SignedHeader_2 = &SignedBeaconBlockHeader{} + } + if err := m.SignedHeader_2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ProposerSlashingV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ProposerSlashingV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ProposerSlashingV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SignedHeader_1", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SignedHeader_1 == nil { + m.SignedHeader_1 = &SignedBeaconBlockHeaderV2{} + } + if err := m.SignedHeader_1.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SignedHeader_2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SignedHeader_2 == nil { + m.SignedHeader_2 = &SignedBeaconBlockHeaderV2{} + } + if err := m.SignedHeader_2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AttesterSlashing) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AttesterSlashing: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AttesterSlashing: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestation_1", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Attestation_1 == nil { + m.Attestation_1 = &IndexedAttestation{} + } + if err := m.Attestation_1.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestation_2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Attestation_2 == nil { + m.Attestation_2 = &IndexedAttestation{} + } + if err := m.Attestation_2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AttesterSlashingV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AttesterSlashingV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AttesterSlashingV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestation_1", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Attestation_1 == nil { + m.Attestation_1 = &IndexedAttestationV2{} + } + if err := m.Attestation_1.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestation_2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Attestation_2 == nil { + m.Attestation_2 = &IndexedAttestationV2{} + } + if err := m.Attestation_2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Deposit_Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Deposit_Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Deposit_Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pubkey", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Pubkey = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithdrawalCredentials", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithdrawalCredentials = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Amount", wireType) + } + m.Amount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Amount |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Deposit) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Deposit: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Deposit: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Proof", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Proof = append(m.Proof, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &Deposit_Data{} + } + if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DepositV2_Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DepositV2_Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DepositV2_Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pubkey", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Pubkey = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithdrawalCredentials", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithdrawalCredentials = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Amount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Amount == nil { + m.Amount = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Amount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DepositV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DepositV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DepositV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Proof", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Proof = append(m.Proof, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &DepositV2_Data{} + } + if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *VoluntaryExit) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: VoluntaryExit: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: VoluntaryExit: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + m.Epoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Epoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + m.ValidatorIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ValidatorIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *VoluntaryExitV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: VoluntaryExitV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: VoluntaryExitV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Epoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidatorIndex == nil { + m.ValidatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedVoluntaryExit) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedVoluntaryExit: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedVoluntaryExit: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &VoluntaryExit{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedVoluntaryExitV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedVoluntaryExitV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedVoluntaryExitV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &VoluntaryExitV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Eth1Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Eth1Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Eth1Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DepositRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DepositRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DepositCount", wireType) + } + m.DepositCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DepositCount |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockHeader) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockHeader: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockHeader: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + m.ProposerIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProposerIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BodyRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BodyRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockHeaderV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockHeaderV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockHeaderV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BodyRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BodyRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockHeader) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockHeader: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockHeader: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockHeader{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockHeaderV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockHeaderV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockHeaderV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockHeaderV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *IndexedAttestation) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: IndexedAttestation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: IndexedAttestation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AttestingIndices = append(m.AttestingIndices, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.AttestingIndices) == 0 { + m.AttestingIndices = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AttestingIndices = append(m.AttestingIndices, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field AttestingIndices", wireType) + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &AttestationData{} + } + if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *IndexedAttestationV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: IndexedAttestationV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: IndexedAttestationV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttestingIndices", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttestingIndices = append(m.AttestingIndices, &wrapperspb1.UInt64Value{}) + if err := (*wrapperspb.UInt64Value)(m.AttestingIndices[len(m.AttestingIndices)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &AttestationDataV2{} + } + if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyncAggregate) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncAggregate: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncAggregate: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncCommitteeBits", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SyncCommitteeBits = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncCommitteeSignature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SyncCommitteeSignature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/blob_vtproto.pb.go b/pkg/proto/eth/v1/blob_vtproto.pb.go new file mode 100644 index 00000000..8667d3fa --- /dev/null +++ b/pkg/proto/eth/v1/blob_vtproto.pb.go @@ -0,0 +1,897 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/blob.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *BlobSidecar) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.BlockParentRoot) > 0 { + i -= len(m.BlockParentRoot) + copy(dAtA[i:], m.BlockParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockParentRoot))) + i-- + dAtA[i] = 0x32 + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x2a + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if len(m.KzgProof) > 0 { + i -= len(m.KzgProof) + copy(dAtA[i:], m.KzgProof) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.KzgProof))) + i-- + dAtA[i] = 0x1a + } + if len(m.KzgCommitment) > 0 { + i -= len(m.KzgCommitment) + copy(dAtA[i:], m.KzgCommitment) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.KzgCommitment))) + i-- + dAtA[i] = 0x12 + } + if len(m.Blob) > 0 { + i -= len(m.Blob) + copy(dAtA[i:], m.Blob) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Blob))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Blob) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Blob) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Blob) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.VersionedHash) > 0 { + i -= len(m.VersionedHash) + copy(dAtA[i:], m.VersionedHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.VersionedHash))) + i-- + dAtA[i] = 0x3a + } + if len(m.KzgCommitment) > 0 { + i -= len(m.KzgCommitment) + copy(dAtA[i:], m.KzgCommitment) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.KzgCommitment))) + i-- + dAtA[i] = 0x32 + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.BlockParentRoot) > 0 { + i -= len(m.BlockParentRoot) + copy(dAtA[i:], m.BlockParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockParentRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x1a + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Blob) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.KzgCommitment) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.KzgProof) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Blob) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.KzgCommitment) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.VersionedHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlobSidecar) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlobSidecar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlobSidecar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Blob", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Blob = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KzgCommitment", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KzgCommitment = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KzgProof", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KzgProof = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Blob) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Blob: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Blob: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KzgCommitment", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KzgCommitment = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionedHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VersionedHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/checkpoint_vtproto.pb.go b/pkg/proto/eth/v1/checkpoint_vtproto.pb.go new file mode 100644 index 00000000..8fc0369a --- /dev/null +++ b/pkg/proto/eth/v1/checkpoint_vtproto.pb.go @@ -0,0 +1,373 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/checkpoint.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *Checkpoint) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Checkpoint) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Checkpoint) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Root) > 0 { + i -= len(m.Root) + copy(dAtA[i:], m.Root) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Root))) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Epoch)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *CheckpointV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CheckpointV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CheckpointV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Root) > 0 { + i -= len(m.Root) + copy(dAtA[i:], m.Root) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Root))) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.Epoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Checkpoint) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Epoch)) + } + l = len(m.Root) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CheckpointV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = (*wrapperspb.UInt64Value)(m.Epoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Root) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Checkpoint) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Checkpoint: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Checkpoint: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + m.Epoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Epoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Root", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Root = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CheckpointV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CheckpointV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CheckpointV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Epoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Root", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Root = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/committee_vtproto.pb.go b/pkg/proto/eth/v1/committee_vtproto.pb.go new file mode 100644 index 00000000..6d399eb0 --- /dev/null +++ b/pkg/proto/eth/v1/committee_vtproto.pb.go @@ -0,0 +1,268 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/committee.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *Committee) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Committee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Committee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Validators) > 0 { + for iNdEx := len(m.Validators) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.UInt64Value)(m.Validators[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Committee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Validators) > 0 { + for _, e := range m.Validators { + l = (*wrapperspb.UInt64Value)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *Committee) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Committee: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Committee: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Validators", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + if err := (*wrapperspb.UInt64Value)(m.Validators[len(m.Validators)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/duties_vtproto.pb.go b/pkg/proto/eth/v1/duties_vtproto.pb.go new file mode 100644 index 00000000..cd63dc35 --- /dev/null +++ b/pkg/proto/eth/v1/duties_vtproto.pb.go @@ -0,0 +1,259 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/duties.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *ProposerDuty) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ProposerDuty) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ProposerDuty) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Pubkey) > 0 { + i -= len(m.Pubkey) + copy(dAtA[i:], m.Pubkey) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Pubkey))) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.ValidatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ProposerDuty) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ValidatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ValidatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Pubkey) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ProposerDuty) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ProposerDuty: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ProposerDuty: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidatorIndex == nil { + m.ValidatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pubkey", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Pubkey = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/events_vtproto.pb.go b/pkg/proto/eth/v1/events_vtproto.pb.go new file mode 100644 index 00000000..4845fc2b --- /dev/null +++ b/pkg/proto/eth/v1/events_vtproto.pb.go @@ -0,0 +1,4753 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/events.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *EventHead) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventHead) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventHead) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.CurrentDutyDependentRoot) > 0 { + i -= len(m.CurrentDutyDependentRoot) + copy(dAtA[i:], m.CurrentDutyDependentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.CurrentDutyDependentRoot))) + i-- + dAtA[i] = 0x32 + } + if len(m.PreviousDutyDependentRoot) > 0 { + i -= len(m.PreviousDutyDependentRoot) + copy(dAtA[i:], m.PreviousDutyDependentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PreviousDutyDependentRoot))) + i-- + dAtA[i] = 0x2a + } + if m.EpochTransition { + i-- + if m.EpochTransition { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 + } + if len(m.State) > 0 { + i -= len(m.State) + copy(dAtA[i:], m.State) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.State))) + i-- + dAtA[i] = 0x1a + } + if len(m.Block) > 0 { + i -= len(m.Block) + copy(dAtA[i:], m.Block) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Block))) + i-- + dAtA[i] = 0x12 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *EventHeadV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventHeadV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventHeadV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.CurrentDutyDependentRoot) > 0 { + i -= len(m.CurrentDutyDependentRoot) + copy(dAtA[i:], m.CurrentDutyDependentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.CurrentDutyDependentRoot))) + i-- + dAtA[i] = 0x32 + } + if len(m.PreviousDutyDependentRoot) > 0 { + i -= len(m.PreviousDutyDependentRoot) + copy(dAtA[i:], m.PreviousDutyDependentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PreviousDutyDependentRoot))) + i-- + dAtA[i] = 0x2a + } + if m.EpochTransition { + i-- + if m.EpochTransition { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 + } + if len(m.State) > 0 { + i -= len(m.State) + copy(dAtA[i:], m.State) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.State))) + i-- + dAtA[i] = 0x1a + } + if len(m.Block) > 0 { + i -= len(m.Block) + copy(dAtA[i:], m.Block) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Block))) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventBlock) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExecutionOptimistic { + i-- + if m.ExecutionOptimistic { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } + if len(m.Block) > 0 { + i -= len(m.Block) + copy(dAtA[i:], m.Block) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Block))) + i-- + dAtA[i] = 0x12 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *EventBlockV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventBlockV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExecutionOptimistic { + i-- + if m.ExecutionOptimistic { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } + if len(m.Block) > 0 { + i -= len(m.Block) + copy(dAtA[i:], m.Block) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Block))) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventChainReorg) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventChainReorg) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventChainReorg) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Epoch)) + i-- + dAtA[i] = 0x38 + } + if len(m.NewHeadState) > 0 { + i -= len(m.NewHeadState) + copy(dAtA[i:], m.NewHeadState) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NewHeadState))) + i-- + dAtA[i] = 0x32 + } + if len(m.OldHeadState) > 0 { + i -= len(m.OldHeadState) + copy(dAtA[i:], m.OldHeadState) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.OldHeadState))) + i-- + dAtA[i] = 0x2a + } + if len(m.NewHeadBlock) > 0 { + i -= len(m.NewHeadBlock) + copy(dAtA[i:], m.NewHeadBlock) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NewHeadBlock))) + i-- + dAtA[i] = 0x22 + } + if len(m.OldHeadBlock) > 0 { + i -= len(m.OldHeadBlock) + copy(dAtA[i:], m.OldHeadBlock) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.OldHeadBlock))) + i-- + dAtA[i] = 0x1a + } + if m.Depth != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Depth)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *EventChainReorgV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventChainReorgV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventChainReorgV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.Epoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.NewHeadState) > 0 { + i -= len(m.NewHeadState) + copy(dAtA[i:], m.NewHeadState) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NewHeadState))) + i-- + dAtA[i] = 0x32 + } + if len(m.OldHeadState) > 0 { + i -= len(m.OldHeadState) + copy(dAtA[i:], m.OldHeadState) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.OldHeadState))) + i-- + dAtA[i] = 0x2a + } + if len(m.NewHeadBlock) > 0 { + i -= len(m.NewHeadBlock) + copy(dAtA[i:], m.NewHeadBlock) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NewHeadBlock))) + i-- + dAtA[i] = 0x22 + } + if len(m.OldHeadBlock) > 0 { + i -= len(m.OldHeadBlock) + copy(dAtA[i:], m.OldHeadBlock) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.OldHeadBlock))) + i-- + dAtA[i] = 0x1a + } + if m.Depth != nil { + size, err := (*wrapperspb.UInt64Value)(m.Depth).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventFinalizedCheckpoint) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventFinalizedCheckpoint) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventFinalizedCheckpoint) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Epoch)) + i-- + dAtA[i] = 0x18 + } + if len(m.State) > 0 { + i -= len(m.State) + copy(dAtA[i:], m.State) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.State))) + i-- + dAtA[i] = 0x12 + } + if len(m.Block) > 0 { + i -= len(m.Block) + copy(dAtA[i:], m.Block) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Block))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventFinalizedCheckpointV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventFinalizedCheckpointV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventFinalizedCheckpointV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.Epoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.State) > 0 { + i -= len(m.State) + copy(dAtA[i:], m.State) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.State))) + i-- + dAtA[i] = 0x12 + } + if len(m.Block) > 0 { + i -= len(m.Block) + copy(dAtA[i:], m.Block) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Block))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventVoluntaryExitMessage) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventVoluntaryExitMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventVoluntaryExitMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ValidatorIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ValidatorIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Epoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Epoch)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *EventVoluntaryExitMessageV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventVoluntaryExitMessageV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventVoluntaryExitMessageV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ValidatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.Epoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventVoluntaryExit) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventVoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventVoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x22 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.ValidatorIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ValidatorIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Epoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Epoch)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *EventVoluntaryExitV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventVoluntaryExitV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventVoluntaryExitV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x22 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} + +func (m *ContributionAndProof) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ContributionAndProof) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ContributionAndProof) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.SelectionProof) > 0 { + i -= len(m.SelectionProof) + copy(dAtA[i:], m.SelectionProof) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.SelectionProof))) + i-- + dAtA[i] = 0x1a + } + if m.Contribution != nil { + size, err := m.Contribution.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.AggregatorIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.AggregatorIndex)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ContributionAndProofV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ContributionAndProofV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ContributionAndProofV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.SelectionProof) > 0 { + i -= len(m.SelectionProof) + copy(dAtA[i:], m.SelectionProof) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.SelectionProof))) + i-- + dAtA[i] = 0x1a + } + if m.Contribution != nil { + size, err := m.Contribution.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.AggregatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.AggregatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventContributionAndProof) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventContributionAndProof) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventContributionAndProof) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventContributionAndProofV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventContributionAndProofV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventContributionAndProofV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventBlobSidecar) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.VersionedHash) > 0 { + i -= len(m.VersionedHash) + copy(dAtA[i:], m.VersionedHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.VersionedHash))) + i-- + dAtA[i] = 0x2a + } + if len(m.KzgCommitment) > 0 { + i -= len(m.KzgCommitment) + copy(dAtA[i:], m.KzgCommitment) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.KzgCommitment))) + i-- + dAtA[i] = 0x22 + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventBlockGossip) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventBlockGossip) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockGossip) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Block) > 0 { + i -= len(m.Block) + copy(dAtA[i:], m.Block) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Block))) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventDataColumnSidecar) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventDataColumnSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventDataColumnSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.KzgCommitmentsCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.KzgCommitmentsCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.KzgCommitments) > 0 { + for iNdEx := len(m.KzgCommitments) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.KzgCommitments[iNdEx]) + copy(dAtA[i:], m.KzgCommitments[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.KzgCommitments[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x1a + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *EventHead) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + l = len(m.Block) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.State) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.EpochTransition { + n += 2 + } + l = len(m.PreviousDutyDependentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.CurrentDutyDependentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventHeadV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Block) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.State) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.EpochTransition { + n += 2 + } + l = len(m.PreviousDutyDependentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.CurrentDutyDependentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + l = len(m.Block) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionOptimistic { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *EventBlockV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Block) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionOptimistic { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *EventChainReorg) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.Depth != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Depth)) + } + l = len(m.OldHeadBlock) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.NewHeadBlock) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.OldHeadState) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.NewHeadState) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Epoch)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventChainReorgV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Depth != nil { + l = (*wrapperspb.UInt64Value)(m.Depth).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.OldHeadBlock) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.NewHeadBlock) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.OldHeadState) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.NewHeadState) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = (*wrapperspb.UInt64Value)(m.Epoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventFinalizedCheckpoint) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Block) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.State) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Epoch)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventFinalizedCheckpointV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Block) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.State) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = (*wrapperspb.UInt64Value)(m.Epoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventVoluntaryExitMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Epoch)) + } + if m.ValidatorIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ValidatorIndex)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventVoluntaryExitMessageV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = (*wrapperspb.UInt64Value)(m.Epoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ValidatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ValidatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventVoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Epoch)) + } + if m.ValidatorIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ValidatorIndex)) + } + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventVoluntaryExitV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ContributionAndProof) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AggregatorIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.AggregatorIndex)) + } + if m.Contribution != nil { + l = m.Contribution.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.SelectionProof) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ContributionAndProofV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AggregatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.AggregatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Contribution != nil { + l = m.Contribution.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.SelectionProof) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventContributionAndProof) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventContributionAndProofV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.KzgCommitment) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.VersionedHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventBlockGossip) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Block) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventDataColumnSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.KzgCommitments) > 0 { + for _, s := range m.KzgCommitments { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.KzgCommitmentsCount != nil { + l = (*wrapperspb.UInt32Value)(m.KzgCommitmentsCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventHead) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventHead: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventHead: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Block = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.State = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EpochTransition", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.EpochTransition = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PreviousDutyDependentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PreviousDutyDependentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CurrentDutyDependentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CurrentDutyDependentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventHeadV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventHeadV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventHeadV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Block = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.State = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EpochTransition", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.EpochTransition = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PreviousDutyDependentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PreviousDutyDependentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CurrentDutyDependentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CurrentDutyDependentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventBlock) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventBlock: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventBlock: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Block = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionOptimistic", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ExecutionOptimistic = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventBlockV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventBlockV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Block = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionOptimistic", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ExecutionOptimistic = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventChainReorg) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventChainReorg: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventChainReorg: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Depth", wireType) + } + m.Depth = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Depth |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OldHeadBlock", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OldHeadBlock = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewHeadBlock", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NewHeadBlock = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OldHeadState", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OldHeadState = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewHeadState", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NewHeadState = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + m.Epoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Epoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventChainReorgV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventChainReorgV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventChainReorgV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Depth", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Depth == nil { + m.Depth = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Depth).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OldHeadBlock", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OldHeadBlock = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewHeadBlock", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NewHeadBlock = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OldHeadState", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OldHeadState = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewHeadState", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NewHeadState = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Epoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventFinalizedCheckpoint) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventFinalizedCheckpoint: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventFinalizedCheckpoint: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Block = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.State = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + m.Epoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Epoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventFinalizedCheckpointV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventFinalizedCheckpointV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventFinalizedCheckpointV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Block = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.State = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Epoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventVoluntaryExitMessage) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventVoluntaryExitMessage: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventVoluntaryExitMessage: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + m.Epoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Epoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + m.ValidatorIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ValidatorIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventVoluntaryExitMessageV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventVoluntaryExitMessageV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventVoluntaryExitMessageV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Epoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidatorIndex == nil { + m.ValidatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventVoluntaryExit) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventVoluntaryExit: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventVoluntaryExit: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + m.Epoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Epoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + m.ValidatorIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ValidatorIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &EventVoluntaryExitMessage{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventVoluntaryExitV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventVoluntaryExitV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventVoluntaryExitV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &EventVoluntaryExitMessageV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ContributionAndProof) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ContributionAndProof: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ContributionAndProof: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregatorIndex", wireType) + } + m.AggregatorIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.AggregatorIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Contribution", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Contribution == nil { + m.Contribution = &SyncCommitteeContribution{} + } + if err := m.Contribution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SelectionProof", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SelectionProof = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ContributionAndProofV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ContributionAndProofV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ContributionAndProofV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AggregatorIndex == nil { + m.AggregatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.AggregatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Contribution", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Contribution == nil { + m.Contribution = &SyncCommitteeContributionV2{} + } + if err := m.Contribution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SelectionProof", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SelectionProof = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventContributionAndProof) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventContributionAndProof: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventContributionAndProof: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &ContributionAndProof{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventContributionAndProofV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventContributionAndProofV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventContributionAndProofV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &ContributionAndProofV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventBlobSidecar) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventBlobSidecar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventBlobSidecar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KzgCommitment", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KzgCommitment = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionedHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VersionedHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventBlockGossip) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventBlockGossip: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventBlockGossip: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Block = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventDataColumnSidecar) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventDataColumnSidecar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventDataColumnSidecar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KzgCommitments", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KzgCommitments = append(m.KzgCommitments, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KzgCommitmentsCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.KzgCommitmentsCount == nil { + m.KzgCommitmentsCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.KzgCommitmentsCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/execution_engine_vtproto.pb.go b/pkg/proto/eth/v1/execution_engine_vtproto.pb.go new file mode 100644 index 00000000..b027997a --- /dev/null +++ b/pkg/proto/eth/v1/execution_engine_vtproto.pb.go @@ -0,0 +1,9580 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/execution_engine.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *ExecutionPayload) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayload) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayload) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Transactions) > 0 { + for iNdEx := len(m.Transactions) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Transactions[iNdEx]) + copy(dAtA[i:], m.Transactions[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Transactions[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x50 + } + if m.GasUsed != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.GasUsed)) + i-- + dAtA[i] = 0x48 + } + if m.GasLimit != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.GasLimit)) + i-- + dAtA[i] = 0x40 + } + if m.BlockNumber != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.BlockNumber)) + i-- + dAtA[i] = 0x38 + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Transactions) > 0 { + for iNdEx := len(m.Transactions) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Transactions[iNdEx]) + copy(dAtA[i:], m.Transactions[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Transactions[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.UInt64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadHeader) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadHeader) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadHeader) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.TransactionsRoot) > 0 { + i -= len(m.TransactionsRoot) + copy(dAtA[i:], m.TransactionsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.TransactionsRoot))) + i-- + dAtA[i] = 0x72 + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x50 + } + if m.GasUsed != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.GasUsed)) + i-- + dAtA[i] = 0x48 + } + if m.GasLimit != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.GasLimit)) + i-- + dAtA[i] = 0x40 + } + if m.BlockNumber != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.BlockNumber)) + i-- + dAtA[i] = 0x38 + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadHeaderV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadHeaderV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadHeaderV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.TransactionsRoot) > 0 { + i -= len(m.TransactionsRoot) + copy(dAtA[i:], m.TransactionsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.TransactionsRoot))) + i-- + dAtA[i] = 0x72 + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.UInt64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadCapella) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadCapella) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadCapella) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Withdrawals) > 0 { + for iNdEx := len(m.Withdrawals) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Withdrawals[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + } + if len(m.Transactions) > 0 { + for iNdEx := len(m.Transactions) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Transactions[iNdEx]) + copy(dAtA[i:], m.Transactions[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Transactions[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x50 + } + if m.GasUsed != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.GasUsed)) + i-- + dAtA[i] = 0x48 + } + if m.GasLimit != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.GasLimit)) + i-- + dAtA[i] = 0x40 + } + if m.BlockNumber != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.BlockNumber)) + i-- + dAtA[i] = 0x38 + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadCapellaV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadCapellaV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadCapellaV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Withdrawals) > 0 { + for iNdEx := len(m.Withdrawals) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Withdrawals[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + } + if len(m.Transactions) > 0 { + for iNdEx := len(m.Transactions) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Transactions[iNdEx]) + copy(dAtA[i:], m.Transactions[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Transactions[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.UInt64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadDeneb) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadDeneb) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadDeneb) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExcessBlobGas != nil { + size, err := (*wrapperspb.UInt64Value)(m.ExcessBlobGas).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + if m.BlobGasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlobGasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if len(m.Withdrawals) > 0 { + for iNdEx := len(m.Withdrawals) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Withdrawals[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + } + if len(m.Transactions) > 0 { + for iNdEx := len(m.Transactions) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Transactions[iNdEx]) + copy(dAtA[i:], m.Transactions[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Transactions[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.UInt64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadElectra) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadElectra) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadElectra) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExcessBlobGas != nil { + size, err := (*wrapperspb.UInt64Value)(m.ExcessBlobGas).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + if m.BlobGasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlobGasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if len(m.Withdrawals) > 0 { + for iNdEx := len(m.Withdrawals) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Withdrawals[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + } + if len(m.Transactions) > 0 { + for iNdEx := len(m.Transactions) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Transactions[iNdEx]) + copy(dAtA[i:], m.Transactions[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Transactions[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.UInt64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadFulu) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadFulu) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadFulu) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExcessBlobGas != nil { + size, err := (*wrapperspb.UInt64Value)(m.ExcessBlobGas).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + if m.BlobGasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlobGasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if len(m.Withdrawals) > 0 { + for iNdEx := len(m.Withdrawals) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Withdrawals[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + } + if len(m.Transactions) > 0 { + for iNdEx := len(m.Transactions) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Transactions[iNdEx]) + copy(dAtA[i:], m.Transactions[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Transactions[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.UInt64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Withdrawal) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Withdrawal) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Withdrawal) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Amount != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Amount)) + i-- + dAtA[i] = 0x20 + } + if len(m.Address) > 0 { + i -= len(m.Address) + copy(dAtA[i:], m.Address) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Address))) + i-- + dAtA[i] = 0x1a + } + if m.ValidatorIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ValidatorIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Index != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Index)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *WithdrawalV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *WithdrawalV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WithdrawalV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Amount != nil { + size, err := (*wrapperspb.UInt64Value)(m.Amount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if len(m.Address) > 0 { + i -= len(m.Address) + copy(dAtA[i:], m.Address) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Address))) + i-- + dAtA[i] = 0x1a + } + if m.ValidatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadHeaderCapella) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadHeaderCapella) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadHeaderCapella) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.WithdrawalsRoot) > 0 { + i -= len(m.WithdrawalsRoot) + copy(dAtA[i:], m.WithdrawalsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.WithdrawalsRoot))) + i-- + dAtA[i] = 0x7a + } + if len(m.TransactionsRoot) > 0 { + i -= len(m.TransactionsRoot) + copy(dAtA[i:], m.TransactionsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.TransactionsRoot))) + i-- + dAtA[i] = 0x72 + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x50 + } + if m.GasUsed != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.GasUsed)) + i-- + dAtA[i] = 0x48 + } + if m.GasLimit != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.GasLimit)) + i-- + dAtA[i] = 0x40 + } + if m.BlockNumber != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.BlockNumber)) + i-- + dAtA[i] = 0x38 + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayloadHeaderCapellaV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionPayloadHeaderCapellaV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionPayloadHeaderCapellaV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.WithdrawalsRoot) > 0 { + i -= len(m.WithdrawalsRoot) + copy(dAtA[i:], m.WithdrawalsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.WithdrawalsRoot))) + i-- + dAtA[i] = 0x7a + } + if len(m.TransactionsRoot) > 0 { + i -= len(m.TransactionsRoot) + copy(dAtA[i:], m.TransactionsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.TransactionsRoot))) + i-- + dAtA[i] = 0x72 + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x6a + } + if len(m.BaseFeePerGas) > 0 { + i -= len(m.BaseFeePerGas) + copy(dAtA[i:], m.BaseFeePerGas) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BaseFeePerGas))) + i-- + dAtA[i] = 0x62 + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x5a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.UInt64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.PrevRandao) > 0 { + i -= len(m.PrevRandao) + copy(dAtA[i:], m.PrevRandao) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PrevRandao))) + i-- + dAtA[i] = 0x32 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x2a + } + if len(m.ReceiptsRoot) > 0 { + i -= len(m.ReceiptsRoot) + copy(dAtA[i:], m.ReceiptsRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ReceiptsRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.FeeRecipient) > 0 { + i -= len(m.FeeRecipient) + copy(dAtA[i:], m.FeeRecipient) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FeeRecipient))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Transaction) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Transaction) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Transaction) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlobHashes) > 0 { + for iNdEx := len(m.BlobHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.BlobHashes[iNdEx]) + copy(dAtA[i:], m.BlobHashes[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobHashes[iNdEx]))) + i-- + dAtA[i] = 0x7a + } + } + if len(m.BlobGasFeeCap) > 0 { + i -= len(m.BlobGasFeeCap) + copy(dAtA[i:], m.BlobGasFeeCap) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobGasFeeCap))) + i-- + dAtA[i] = 0x72 + } + if m.BlobGas != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlobGas).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if len(m.GasFeeCap) > 0 { + i -= len(m.GasFeeCap) + copy(dAtA[i:], m.GasFeeCap) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.GasFeeCap))) + i-- + dAtA[i] = 0x62 + } + if len(m.GasTipCap) > 0 { + i -= len(m.GasTipCap) + copy(dAtA[i:], m.GasTipCap) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.GasTipCap))) + i-- + dAtA[i] = 0x5a + } + if m.Type != nil { + size, err := (*wrapperspb.UInt32Value)(m.Type).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x4a + } + if m.Nonce != nil { + size, err := (*wrapperspb.UInt64Value)(m.Nonce).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if len(m.To) > 0 { + i -= len(m.To) + copy(dAtA[i:], m.To) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.To))) + i-- + dAtA[i] = 0x3a + } + if len(m.From) > 0 { + i -= len(m.From) + copy(dAtA[i:], m.From) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.From))) + i-- + dAtA[i] = 0x32 + } + if len(m.Hash) > 0 { + i -= len(m.Hash) + copy(dAtA[i:], m.Hash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Hash))) + i-- + dAtA[i] = 0x2a + } + if len(m.GasPrice) > 0 { + i -= len(m.GasPrice) + copy(dAtA[i:], m.GasPrice) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.GasPrice))) + i-- + dAtA[i] = 0x22 + } + if m.Gas != nil { + size, err := (*wrapperspb.UInt64Value)(m.Gas).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.Input) > 0 { + i -= len(m.Input) + copy(dAtA[i:], m.Input) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Input))) + i-- + dAtA[i] = 0x12 + } + if len(m.ChainId) > 0 { + i -= len(m.ChainId) + copy(dAtA[i:], m.ChainId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ChainId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionPayload) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.BlockNumber)) + } + if m.GasLimit != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.GasLimit)) + } + if m.GasUsed != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.GasUsed)) + } + if m.Timestamp != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Timestamp)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Transactions) > 0 { + for _, s := range m.Transactions { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.UInt64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Transactions) > 0 { + for _, s := range m.Transactions { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadHeader) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.BlockNumber)) + } + if m.GasLimit != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.GasLimit)) + } + if m.GasUsed != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.GasUsed)) + } + if m.Timestamp != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Timestamp)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.TransactionsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadHeaderV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.UInt64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.TransactionsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadCapella) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.BlockNumber)) + } + if m.GasLimit != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.GasLimit)) + } + if m.GasUsed != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.GasUsed)) + } + if m.Timestamp != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Timestamp)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Transactions) > 0 { + for _, s := range m.Transactions { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Withdrawals) > 0 { + for _, e := range m.Withdrawals { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadCapellaV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.UInt64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Transactions) > 0 { + for _, s := range m.Transactions { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Withdrawals) > 0 { + for _, e := range m.Withdrawals { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadDeneb) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.UInt64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Transactions) > 0 { + for _, s := range m.Transactions { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Withdrawals) > 0 { + for _, e := range m.Withdrawals { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.BlobGasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.BlobGasUsed).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExcessBlobGas != nil { + l = (*wrapperspb.UInt64Value)(m.ExcessBlobGas).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadElectra) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.UInt64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Transactions) > 0 { + for _, s := range m.Transactions { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Withdrawals) > 0 { + for _, e := range m.Withdrawals { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.BlobGasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.BlobGasUsed).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExcessBlobGas != nil { + l = (*wrapperspb.UInt64Value)(m.ExcessBlobGas).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadFulu) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.UInt64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Transactions) > 0 { + for _, s := range m.Transactions { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Withdrawals) > 0 { + for _, e := range m.Withdrawals { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.BlobGasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.BlobGasUsed).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExcessBlobGas != nil { + l = (*wrapperspb.UInt64Value)(m.ExcessBlobGas).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Withdrawal) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Index)) + } + if m.ValidatorIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ValidatorIndex)) + } + l = len(m.Address) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Amount != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Amount)) + } + n += len(m.unknownFields) + return n +} + +func (m *WithdrawalV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ValidatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ValidatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Address) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Amount != nil { + l = (*wrapperspb.UInt64Value)(m.Amount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadHeaderCapella) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.BlockNumber)) + } + if m.GasLimit != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.GasLimit)) + } + if m.GasUsed != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.GasUsed)) + } + if m.Timestamp != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Timestamp)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.TransactionsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.WithdrawalsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayloadHeaderCapellaV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FeeRecipient) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ReceiptsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PrevRandao) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.UInt64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BaseFeePerGas) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.TransactionsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.WithdrawalsRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Transaction) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ChainId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Input) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Gas != nil { + l = (*wrapperspb.UInt64Value)(m.Gas).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.GasPrice) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Hash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.From) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.To) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Nonce != nil { + l = (*wrapperspb.UInt64Value)(m.Nonce).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Type != nil { + l = (*wrapperspb.UInt32Value)(m.Type).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.GasTipCap) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.GasFeeCap) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlobGas != nil { + l = (*wrapperspb.UInt64Value)(m.BlobGas).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlobGasFeeCap) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlobHashes) > 0 { + for _, s := range m.BlobHashes { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionPayload) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayload: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayload: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + m.BlockNumber = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BlockNumber |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + m.GasLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasLimit |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + m.GasUsed = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasUsed |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transactions", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transactions = append(m.Transactions, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transactions", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transactions = append(m.Transactions, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadHeader) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadHeader: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadHeader: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + m.BlockNumber = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BlockNumber |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + m.GasLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasLimit |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + m.GasUsed = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasUsed |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TransactionsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadHeaderV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadHeaderV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadHeaderV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TransactionsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadCapella) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadCapella: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadCapella: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + m.BlockNumber = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BlockNumber |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + m.GasLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasLimit |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + m.GasUsed = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasUsed |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transactions", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transactions = append(m.Transactions, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Withdrawals", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Withdrawals = append(m.Withdrawals, &Withdrawal{}) + if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadCapellaV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadCapellaV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadCapellaV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transactions", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transactions = append(m.Transactions, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Withdrawals", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadDeneb) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadDeneb: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadDeneb: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transactions", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transactions = append(m.Transactions, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Withdrawals", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobGasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlobGasUsed == nil { + m.BlobGasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlobGasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExcessBlobGas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExcessBlobGas == nil { + m.ExcessBlobGas = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ExcessBlobGas).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadElectra) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadElectra: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadElectra: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transactions", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transactions = append(m.Transactions, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Withdrawals", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobGasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlobGasUsed == nil { + m.BlobGasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlobGasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExcessBlobGas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExcessBlobGas == nil { + m.ExcessBlobGas = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ExcessBlobGas).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadFulu) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadFulu: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadFulu: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transactions", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transactions = append(m.Transactions, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Withdrawals", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobGasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlobGasUsed == nil { + m.BlobGasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlobGasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExcessBlobGas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExcessBlobGas == nil { + m.ExcessBlobGas = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ExcessBlobGas).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Withdrawal) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Withdrawal: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Withdrawal: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + m.Index = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Index |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + m.ValidatorIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ValidatorIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Address", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Address = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Amount", wireType) + } + m.Amount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Amount |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *WithdrawalV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: WithdrawalV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WithdrawalV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidatorIndex == nil { + m.ValidatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Address", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Address = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Amount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Amount == nil { + m.Amount = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Amount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadHeaderCapella) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadHeaderCapella: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadHeaderCapella: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + m.BlockNumber = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BlockNumber |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + m.GasLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasLimit |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + m.GasUsed = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasUsed |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TransactionsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithdrawalsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithdrawalsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionPayloadHeaderCapellaV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionPayloadHeaderCapellaV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionPayloadHeaderCapellaV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FeeRecipient = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrevRandao", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrevRandao = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFeePerGas", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFeePerGas = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TransactionsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithdrawalsRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithdrawalsRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Transaction) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Transaction: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Transaction: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ChainId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ChainId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Input", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Input = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Gas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Gas == nil { + m.Gas = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Gas).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasPrice", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.GasPrice = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field From", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.From = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field To", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.To = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Nonce", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Nonce == nil { + m.Nonce = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Nonce).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Type == nil { + m.Type = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Type).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasTipCap", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.GasTipCap = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasFeeCap", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.GasFeeCap = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobGas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlobGas == nil { + m.BlobGas = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlobGas).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobGasFeeCap", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobGasFeeCap = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobHashes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobHashes = append(m.BlobHashes, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/execution_requests_vtproto.pb.go b/pkg/proto/eth/v1/execution_requests_vtproto.pb.go new file mode 100644 index 00000000..1e87ae65 --- /dev/null +++ b/pkg/proto/eth/v1/execution_requests_vtproto.pb.go @@ -0,0 +1,1104 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/execution_requests.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *ElectraExecutionRequests) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ElectraExecutionRequests) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ElectraExecutionRequests) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Consolidations) > 0 { + for iNdEx := len(m.Consolidations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Consolidations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if len(m.Withdrawals) > 0 { + for iNdEx := len(m.Withdrawals) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Withdrawals[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *ElectraExecutionRequestDeposit) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ElectraExecutionRequestDeposit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ElectraExecutionRequestDeposit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Signature != nil { + size, err := (*wrapperspb.StringValue)(m.Signature).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Amount != nil { + size, err := (*wrapperspb.UInt64Value)(m.Amount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.WithdrawalCredentials != nil { + size, err := (*wrapperspb.StringValue)(m.WithdrawalCredentials).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Pubkey != nil { + size, err := (*wrapperspb.StringValue)(m.Pubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ElectraExecutionRequestWithdrawal) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ElectraExecutionRequestWithdrawal) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ElectraExecutionRequestWithdrawal) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Amount != nil { + size, err := (*wrapperspb.UInt64Value)(m.Amount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.ValidatorPubkey != nil { + size, err := (*wrapperspb.StringValue)(m.ValidatorPubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.SourceAddress != nil { + size, err := (*wrapperspb.StringValue)(m.SourceAddress).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ElectraExecutionRequestConsolidation) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ElectraExecutionRequestConsolidation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ElectraExecutionRequestConsolidation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.TargetPubkey != nil { + size, err := (*wrapperspb.StringValue)(m.TargetPubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.SourcePubkey != nil { + size, err := (*wrapperspb.StringValue)(m.SourcePubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.SourceAddress != nil { + size, err := (*wrapperspb.StringValue)(m.SourceAddress).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ElectraExecutionRequests) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Withdrawals) > 0 { + for _, e := range m.Withdrawals { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Consolidations) > 0 { + for _, e := range m.Consolidations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ElectraExecutionRequestDeposit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Pubkey != nil { + l = (*wrapperspb.StringValue)(m.Pubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WithdrawalCredentials != nil { + l = (*wrapperspb.StringValue)(m.WithdrawalCredentials).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Amount != nil { + l = (*wrapperspb.UInt64Value)(m.Amount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Signature != nil { + l = (*wrapperspb.StringValue)(m.Signature).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ElectraExecutionRequestWithdrawal) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SourceAddress != nil { + l = (*wrapperspb.StringValue)(m.SourceAddress).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ValidatorPubkey != nil { + l = (*wrapperspb.StringValue)(m.ValidatorPubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Amount != nil { + l = (*wrapperspb.UInt64Value)(m.Amount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ElectraExecutionRequestConsolidation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SourceAddress != nil { + l = (*wrapperspb.StringValue)(m.SourceAddress).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SourcePubkey != nil { + l = (*wrapperspb.StringValue)(m.SourcePubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TargetPubkey != nil { + l = (*wrapperspb.StringValue)(m.TargetPubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ElectraExecutionRequests) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ElectraExecutionRequests: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ElectraExecutionRequests: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &ElectraExecutionRequestDeposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Withdrawals", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Withdrawals = append(m.Withdrawals, &ElectraExecutionRequestWithdrawal{}) + if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Consolidations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Consolidations = append(m.Consolidations, &ElectraExecutionRequestConsolidation{}) + if err := m.Consolidations[len(m.Consolidations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ElectraExecutionRequestDeposit) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ElectraExecutionRequestDeposit: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ElectraExecutionRequestDeposit: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Pubkey == nil { + m.Pubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Pubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithdrawalCredentials", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WithdrawalCredentials == nil { + m.WithdrawalCredentials = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.WithdrawalCredentials).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Amount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Amount == nil { + m.Amount = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Amount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Signature == nil { + m.Signature = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Signature).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ElectraExecutionRequestWithdrawal) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ElectraExecutionRequestWithdrawal: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ElectraExecutionRequestWithdrawal: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceAddress", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SourceAddress == nil { + m.SourceAddress = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.SourceAddress).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorPubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidatorPubkey == nil { + m.ValidatorPubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ValidatorPubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Amount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Amount == nil { + m.Amount = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Amount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ElectraExecutionRequestConsolidation) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ElectraExecutionRequestConsolidation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ElectraExecutionRequestConsolidation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceAddress", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SourceAddress == nil { + m.SourceAddress = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.SourceAddress).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourcePubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SourcePubkey == nil { + m.SourcePubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.SourcePubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TargetPubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TargetPubkey == nil { + m.TargetPubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TargetPubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/fork_choice_vtproto.pb.go b/pkg/proto/eth/v1/fork_choice_vtproto.pb.go new file mode 100644 index 00000000..1ed0f1aa --- /dev/null +++ b/pkg/proto/eth/v1/fork_choice_vtproto.pb.go @@ -0,0 +1,1440 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/fork_choice.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *ForkChoice) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ForkChoice) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ForkChoice) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ForkChoiceNodes) > 0 { + for iNdEx := len(m.ForkChoiceNodes) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ForkChoiceNodes[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if m.FinalizedCheckpoint != nil { + size, err := m.FinalizedCheckpoint.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.JustifiedCheckpoint != nil { + size, err := m.JustifiedCheckpoint.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ForkChoiceV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ForkChoiceV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ForkChoiceV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ForkChoiceNodes) > 0 { + for iNdEx := len(m.ForkChoiceNodes) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ForkChoiceNodes[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if m.FinalizedCheckpoint != nil { + size, err := m.FinalizedCheckpoint.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.JustifiedCheckpoint != nil { + size, err := m.JustifiedCheckpoint.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ForkChoiceNode) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ForkChoiceNode) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ForkChoiceNode) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x4a + } + if len(m.ExecutionBlockHash) > 0 { + i -= len(m.ExecutionBlockHash) + copy(dAtA[i:], m.ExecutionBlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExecutionBlockHash))) + i-- + dAtA[i] = 0x42 + } + if len(m.Validity) > 0 { + i -= len(m.Validity) + copy(dAtA[i:], m.Validity) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Validity))) + i-- + dAtA[i] = 0x3a + } + if m.Weight != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Weight)) + i-- + dAtA[i] = 0x30 + } + if m.FinalizedEpoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.FinalizedEpoch)) + i-- + dAtA[i] = 0x28 + } + if m.JustifiedEpoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.JustifiedEpoch)) + i-- + dAtA[i] = 0x20 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x12 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ForkChoiceNodeV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ForkChoiceNodeV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ForkChoiceNodeV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ExtraData) > 0 { + i -= len(m.ExtraData) + copy(dAtA[i:], m.ExtraData) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExtraData))) + i-- + dAtA[i] = 0x4a + } + if len(m.ExecutionBlockHash) > 0 { + i -= len(m.ExecutionBlockHash) + copy(dAtA[i:], m.ExecutionBlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExecutionBlockHash))) + i-- + dAtA[i] = 0x42 + } + if len(m.Validity) > 0 { + i -= len(m.Validity) + copy(dAtA[i:], m.Validity) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Validity))) + i-- + dAtA[i] = 0x3a + } + if m.Weight != nil { + size, err := (*wrapperspb.UInt64Value)(m.Weight).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.FinalizedEpoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.FinalizedEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.JustifiedEpoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.JustifiedEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ForkChoice) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.JustifiedCheckpoint != nil { + l = m.JustifiedCheckpoint.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedCheckpoint != nil { + l = m.FinalizedCheckpoint.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ForkChoiceNodes) > 0 { + for _, e := range m.ForkChoiceNodes { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ForkChoiceV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.JustifiedCheckpoint != nil { + l = m.JustifiedCheckpoint.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedCheckpoint != nil { + l = m.FinalizedCheckpoint.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ForkChoiceNodes) > 0 { + for _, e := range m.ForkChoiceNodes { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ForkChoiceNode) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.JustifiedEpoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.JustifiedEpoch)) + } + if m.FinalizedEpoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.FinalizedEpoch)) + } + if m.Weight != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Weight)) + } + l = len(m.Validity) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExecutionBlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ForkChoiceNodeV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.JustifiedEpoch != nil { + l = (*wrapperspb.UInt64Value)(m.JustifiedEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedEpoch != nil { + l = (*wrapperspb.UInt64Value)(m.FinalizedEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Weight != nil { + l = (*wrapperspb.UInt64Value)(m.Weight).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Validity) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExecutionBlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExtraData) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ForkChoice) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ForkChoice: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ForkChoice: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field JustifiedCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.JustifiedCheckpoint == nil { + m.JustifiedCheckpoint = &Checkpoint{} + } + if err := m.JustifiedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedCheckpoint == nil { + m.FinalizedCheckpoint = &Checkpoint{} + } + if err := m.FinalizedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkChoiceNodes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ForkChoiceNodes = append(m.ForkChoiceNodes, &ForkChoiceNode{}) + if err := m.ForkChoiceNodes[len(m.ForkChoiceNodes)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ForkChoiceV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ForkChoiceV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ForkChoiceV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field JustifiedCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.JustifiedCheckpoint == nil { + m.JustifiedCheckpoint = &CheckpointV2{} + } + if err := m.JustifiedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedCheckpoint == nil { + m.FinalizedCheckpoint = &CheckpointV2{} + } + if err := m.FinalizedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkChoiceNodes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ForkChoiceNodes = append(m.ForkChoiceNodes, &ForkChoiceNodeV2{}) + if err := m.ForkChoiceNodes[len(m.ForkChoiceNodes)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ForkChoiceNode) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ForkChoiceNode: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ForkChoiceNode: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JustifiedEpoch", wireType) + } + m.JustifiedEpoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JustifiedEpoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpoch", wireType) + } + m.FinalizedEpoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FinalizedEpoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Weight", wireType) + } + m.Weight = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Weight |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Validity", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Validity = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionBlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExecutionBlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ForkChoiceNodeV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ForkChoiceNodeV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ForkChoiceNodeV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field JustifiedEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.JustifiedEpoch == nil { + m.JustifiedEpoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.JustifiedEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedEpoch == nil { + m.FinalizedEpoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.FinalizedEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Weight", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Weight == nil { + m.Weight = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Weight).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Validity", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Validity = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionBlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExecutionBlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraData", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraData = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/sync_committee_vtproto.pb.go b/pkg/proto/eth/v1/sync_committee_vtproto.pb.go new file mode 100644 index 00000000..fe64497b --- /dev/null +++ b/pkg/proto/eth/v1/sync_committee_vtproto.pb.go @@ -0,0 +1,966 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/sync_committee.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *SyncCommitteeContribution) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncCommitteeContribution) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SyncCommitteeContribution) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BeaconBlockRoot) > 0 { + i -= len(m.BeaconBlockRoot) + copy(dAtA[i:], m.BeaconBlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BeaconBlockRoot))) + i-- + dAtA[i] = 0x2a + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x22 + } + if len(m.AggregationBits) > 0 { + i -= len(m.AggregationBits) + copy(dAtA[i:], m.AggregationBits) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.AggregationBits))) + i-- + dAtA[i] = 0x1a + } + if m.SubcommitteeIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.SubcommitteeIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SyncCommitteeContributionV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncCommitteeContributionV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SyncCommitteeContributionV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BeaconBlockRoot) > 0 { + i -= len(m.BeaconBlockRoot) + copy(dAtA[i:], m.BeaconBlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BeaconBlockRoot))) + i-- + dAtA[i] = 0x2a + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x22 + } + if len(m.AggregationBits) > 0 { + i -= len(m.AggregationBits) + copy(dAtA[i:], m.AggregationBits) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.AggregationBits))) + i-- + dAtA[i] = 0x1a + } + if m.SubcommitteeIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.SubcommitteeIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SyncCommitteeValidatorAggregate) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncCommitteeValidatorAggregate) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SyncCommitteeValidatorAggregate) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Validators) > 0 { + for iNdEx := len(m.Validators) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.UInt64Value)(m.Validators[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SyncCommittee) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SyncCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ValidatorAggregates) > 0 { + for iNdEx := len(m.ValidatorAggregates) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ValidatorAggregates[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Validators) > 0 { + for iNdEx := len(m.Validators) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.UInt64Value)(m.Validators[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SyncCommitteeContribution) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.SubcommitteeIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.SubcommitteeIndex)) + } + l = len(m.AggregationBits) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BeaconBlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SyncCommitteeContributionV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SubcommitteeIndex != nil { + l = (*wrapperspb.UInt64Value)(m.SubcommitteeIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.AggregationBits) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BeaconBlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SyncCommitteeValidatorAggregate) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Validators) > 0 { + for _, e := range m.Validators { + l = (*wrapperspb.UInt64Value)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *SyncCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Validators) > 0 { + for _, e := range m.Validators { + l = (*wrapperspb.UInt64Value)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.ValidatorAggregates) > 0 { + for _, e := range m.ValidatorAggregates { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *SyncCommitteeContribution) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncCommitteeContribution: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncCommitteeContribution: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SubcommitteeIndex", wireType) + } + m.SubcommitteeIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SubcommitteeIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregationBits", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AggregationBits = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeaconBlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BeaconBlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyncCommitteeContributionV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncCommitteeContributionV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncCommitteeContributionV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SubcommitteeIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SubcommitteeIndex == nil { + m.SubcommitteeIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SubcommitteeIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregationBits", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AggregationBits = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeaconBlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BeaconBlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyncCommitteeValidatorAggregate) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncCommitteeValidatorAggregate: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncCommitteeValidatorAggregate: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Validators", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + if err := (*wrapperspb.UInt64Value)(m.Validators[len(m.Validators)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyncCommittee) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncCommittee: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncCommittee: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Validators", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + if err := (*wrapperspb.UInt64Value)(m.Validators[len(m.Validators)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorAggregates", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ValidatorAggregates = append(m.ValidatorAggregates, &SyncCommitteeValidatorAggregate{}) + if err := m.ValidatorAggregates[len(m.ValidatorAggregates)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v1/validator_vtproto.pb.go b/pkg/proto/eth/v1/validator_vtproto.pb.go new file mode 100644 index 00000000..95f26be9 --- /dev/null +++ b/pkg/proto/eth/v1/validator_vtproto.pb.go @@ -0,0 +1,810 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v1/validator.proto + +package v1 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *ValidatorData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ValidatorData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ValidatorData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.WithdrawableEpoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.WithdrawableEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.ExitEpoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.ExitEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.ActivationEpoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.ActivationEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.ActivationEligibilityEpoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.ActivationEligibilityEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Slashed != nil { + size, err := (*wrapperspb.BoolValue)(m.Slashed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.EffectiveBalance != nil { + size, err := (*wrapperspb.UInt64Value)(m.EffectiveBalance).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.WithdrawalCredentials != nil { + size, err := (*wrapperspb.StringValue)(m.WithdrawalCredentials).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Pubkey != nil { + size, err := (*wrapperspb.StringValue)(m.Pubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Validator) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Validator) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Validator) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Status != nil { + size, err := (*wrapperspb.StringValue)(m.Status).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Balance != nil { + size, err := (*wrapperspb.UInt64Value)(m.Balance).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Data != nil { + size, err := m.Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ValidatorData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Pubkey != nil { + l = (*wrapperspb.StringValue)(m.Pubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WithdrawalCredentials != nil { + l = (*wrapperspb.StringValue)(m.WithdrawalCredentials).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.EffectiveBalance != nil { + l = (*wrapperspb.UInt64Value)(m.EffectiveBalance).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slashed != nil { + l = (*wrapperspb.BoolValue)(m.Slashed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ActivationEligibilityEpoch != nil { + l = (*wrapperspb.UInt64Value)(m.ActivationEligibilityEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ActivationEpoch != nil { + l = (*wrapperspb.UInt64Value)(m.ActivationEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExitEpoch != nil { + l = (*wrapperspb.UInt64Value)(m.ExitEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WithdrawableEpoch != nil { + l = (*wrapperspb.UInt64Value)(m.WithdrawableEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Validator) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Data != nil { + l = m.Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Balance != nil { + l = (*wrapperspb.UInt64Value)(m.Balance).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Status != nil { + l = (*wrapperspb.StringValue)(m.Status).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ValidatorData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ValidatorData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ValidatorData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Pubkey == nil { + m.Pubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Pubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithdrawalCredentials", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WithdrawalCredentials == nil { + m.WithdrawalCredentials = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.WithdrawalCredentials).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EffectiveBalance", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EffectiveBalance == nil { + m.EffectiveBalance = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.EffectiveBalance).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slashed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slashed == nil { + m.Slashed = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Slashed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ActivationEligibilityEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ActivationEligibilityEpoch == nil { + m.ActivationEligibilityEpoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ActivationEligibilityEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ActivationEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ActivationEpoch == nil { + m.ActivationEpoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ActivationEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExitEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExitEpoch == nil { + m.ExitEpoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ExitEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithdrawableEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WithdrawableEpoch == nil { + m.WithdrawableEpoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.WithdrawableEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Validator) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Validator: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Validator: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &ValidatorData{} + } + if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Balance", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Balance == nil { + m.Balance = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Balance).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Status == nil { + m.Status = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Status).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v2/beacon_block_vtproto.pb.go b/pkg/proto/eth/v2/beacon_block_vtproto.pb.go new file mode 100644 index 00000000..6d48d793 --- /dev/null +++ b/pkg/proto/eth/v2/beacon_block_vtproto.pb.go @@ -0,0 +1,14689 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v2/beacon_block.proto + +package v2 + +import ( + fmt "fmt" + v1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *SignedBeaconBlockBellatrix) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockBellatrix) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockBellatrix) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockBellatrixV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockBellatrixV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockBellatrixV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockCapella) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockCapella) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockCapella) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockCapellaV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockCapellaV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockCapellaV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockDeneb) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockDeneb) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockDeneb) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockElectra) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockElectra) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockElectra) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockFulu) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockFulu) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockFulu) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBlindedBeaconBlockBellatrix) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBlindedBeaconBlockBellatrix) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBlindedBeaconBlockBellatrix) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBlindedBeaconBlockBellatrixV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBlindedBeaconBlockBellatrixV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBlindedBeaconBlockBellatrixV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBlindedBeaconBlockCapella) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBlindedBeaconBlockCapella) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBlindedBeaconBlockCapella) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBlindedBeaconBlockCapellaV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBlindedBeaconBlockCapellaV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBlindedBeaconBlockCapellaV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockAltair) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockAltair) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockAltair) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockAltairV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBeaconBlockAltairV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBeaconBlockAltairV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBellatrix) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBellatrix) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBellatrix) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ProposerIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBellatrixV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBellatrixV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBellatrixV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlindedBeaconBlockBellatrix) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlindedBeaconBlockBellatrix) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlindedBeaconBlockBellatrix) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ProposerIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *BlindedBeaconBlockBellatrixV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlindedBeaconBlockBellatrixV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlindedBeaconBlockBellatrixV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockCapella) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockCapella) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockCapella) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ProposerIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockCapellaV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockCapellaV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockCapellaV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockDeneb) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockDeneb) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockDeneb) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockElectra) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockElectra) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockElectra) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockFulu) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockFulu) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockFulu) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlindedBeaconBlockCapella) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlindedBeaconBlockCapella) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlindedBeaconBlockCapella) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ProposerIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *BlindedBeaconBlockCapellaV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlindedBeaconBlockCapellaV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlindedBeaconBlockCapellaV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockAltair) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockAltair) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockAltair) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ProposerIndex)) + i-- + dAtA[i] = 0x10 + } + if m.Slot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Slot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockAltairV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockAltairV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockAltairV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Body != nil { + size, err := m.Body.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.ParentRoot) > 0 { + i -= len(m.ParentRoot) + copy(dAtA[i:], m.ParentRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentRoot))) + i-- + dAtA[i] = 0x1a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyBellatrix) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyBellatrix) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyBellatrix) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExecutionPayload != nil { + size, err := m.ExecutionPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyBellatrixV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyBellatrixV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyBellatrixV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExecutionPayload != nil { + size, err := m.ExecutionPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlindedBeaconBlockBodyBellatrix) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlindedBeaconBlockBodyBellatrix) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlindedBeaconBlockBodyBellatrix) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExecutionPayloadHeader != nil { + size, err := m.ExecutionPayloadHeader.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlindedBeaconBlockBodyBellatrixV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlindedBeaconBlockBodyBellatrixV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlindedBeaconBlockBodyBellatrixV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExecutionPayloadHeader != nil { + size, err := m.ExecutionPayloadHeader.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyCapella) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyCapella) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyCapella) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlsToExecutionChanges) > 0 { + for iNdEx := len(m.BlsToExecutionChanges) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.BlsToExecutionChanges[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + } + if m.ExecutionPayload != nil { + size, err := m.ExecutionPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyCapellaV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyCapellaV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyCapellaV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlsToExecutionChanges) > 0 { + for iNdEx := len(m.BlsToExecutionChanges) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.BlsToExecutionChanges[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + } + if m.ExecutionPayload != nil { + size, err := m.ExecutionPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyDeneb) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyDeneb) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyDeneb) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlobKzgCommitments) > 0 { + for iNdEx := len(m.BlobKzgCommitments) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.BlobKzgCommitments[iNdEx]) + copy(dAtA[i:], m.BlobKzgCommitments[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobKzgCommitments[iNdEx]))) + i-- + dAtA[i] = 0x62 + } + } + if len(m.BlsToExecutionChanges) > 0 { + for iNdEx := len(m.BlsToExecutionChanges) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.BlsToExecutionChanges[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + } + if m.ExecutionPayload != nil { + size, err := m.ExecutionPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyElectra) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyElectra) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyElectra) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExecutionRequests != nil { + size, err := m.ExecutionRequests.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if len(m.BlobKzgCommitments) > 0 { + for iNdEx := len(m.BlobKzgCommitments) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.BlobKzgCommitments[iNdEx]) + copy(dAtA[i:], m.BlobKzgCommitments[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobKzgCommitments[iNdEx]))) + i-- + dAtA[i] = 0x62 + } + } + if len(m.BlsToExecutionChanges) > 0 { + for iNdEx := len(m.BlsToExecutionChanges) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.BlsToExecutionChanges[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + } + if m.ExecutionPayload != nil { + size, err := m.ExecutionPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyFulu) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyFulu) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyFulu) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ExecutionRequests != nil { + size, err := m.ExecutionRequests.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if len(m.BlobKzgCommitments) > 0 { + for iNdEx := len(m.BlobKzgCommitments) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.BlobKzgCommitments[iNdEx]) + copy(dAtA[i:], m.BlobKzgCommitments[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobKzgCommitments[iNdEx]))) + i-- + dAtA[i] = 0x62 + } + } + if len(m.BlsToExecutionChanges) > 0 { + for iNdEx := len(m.BlsToExecutionChanges) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.BlsToExecutionChanges[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + } + if m.ExecutionPayload != nil { + size, err := m.ExecutionPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlindedBeaconBlockBodyCapella) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlindedBeaconBlockBodyCapella) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlindedBeaconBlockBodyCapella) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlsToExecutionChanges) > 0 { + for iNdEx := len(m.BlsToExecutionChanges) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.BlsToExecutionChanges[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + } + if m.ExecutionPayloadHeader != nil { + size, err := m.ExecutionPayloadHeader.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlindedBeaconBlockBodyCapellaV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlindedBeaconBlockBodyCapellaV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlindedBeaconBlockBodyCapellaV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlsToExecutionChanges) > 0 { + for iNdEx := len(m.BlsToExecutionChanges) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.BlsToExecutionChanges[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + } + if m.ExecutionPayloadHeader != nil { + size, err := m.ExecutionPayloadHeader.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyAltair) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyAltair) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyAltair) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlockBodyAltairV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlockBodyAltairV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlockBodyAltairV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SyncAggregate != nil { + size, err := m.SyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.VoluntaryExits) > 0 { + for iNdEx := len(m.VoluntaryExits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.VoluntaryExits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + } + if len(m.Deposits) > 0 { + for iNdEx := len(m.Deposits) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Deposits[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Attestations) > 0 { + for iNdEx := len(m.Attestations) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Attestations[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + } + if len(m.AttesterSlashings) > 0 { + for iNdEx := len(m.AttesterSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.AttesterSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ProposerSlashings) > 0 { + for iNdEx := len(m.ProposerSlashings) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.ProposerSlashings[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graffiti) > 0 { + i -= len(m.Graffiti) + copy(dAtA[i:], m.Graffiti) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Graffiti))) + i-- + dAtA[i] = 0x1a + } + if m.Eth1Data != nil { + size, err := m.Eth1Data.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.RandaoReveal) > 0 { + i -= len(m.RandaoReveal) + copy(dAtA[i:], m.RandaoReveal) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RandaoReveal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBeaconBlockBellatrix) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockBellatrixV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockCapella) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockCapellaV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockDeneb) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockElectra) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockFulu) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBlindedBeaconBlockBellatrix) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBlindedBeaconBlockBellatrixV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBlindedBeaconBlockCapella) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBlindedBeaconBlockCapellaV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockAltair) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockAltairV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBellatrix) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.ProposerIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ProposerIndex)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBellatrixV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlindedBeaconBlockBellatrix) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.ProposerIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ProposerIndex)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlindedBeaconBlockBellatrixV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockCapella) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.ProposerIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ProposerIndex)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockCapellaV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockDeneb) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockElectra) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockFulu) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlindedBeaconBlockCapella) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.ProposerIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ProposerIndex)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlindedBeaconBlockCapellaV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockAltair) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Slot)) + } + if m.ProposerIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ProposerIndex)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockAltairV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Body != nil { + l = m.Body.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyBellatrix) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayload != nil { + l = m.ExecutionPayload.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyBellatrixV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayload != nil { + l = m.ExecutionPayload.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlindedBeaconBlockBodyBellatrix) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayloadHeader != nil { + l = m.ExecutionPayloadHeader.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlindedBeaconBlockBodyBellatrixV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayloadHeader != nil { + l = m.ExecutionPayloadHeader.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyCapella) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayload != nil { + l = m.ExecutionPayload.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlsToExecutionChanges) > 0 { + for _, e := range m.BlsToExecutionChanges { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyCapellaV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayload != nil { + l = m.ExecutionPayload.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlsToExecutionChanges) > 0 { + for _, e := range m.BlsToExecutionChanges { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyDeneb) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayload != nil { + l = m.ExecutionPayload.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlsToExecutionChanges) > 0 { + for _, e := range m.BlsToExecutionChanges { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.BlobKzgCommitments) > 0 { + for _, s := range m.BlobKzgCommitments { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyElectra) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayload != nil { + l = m.ExecutionPayload.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlsToExecutionChanges) > 0 { + for _, e := range m.BlsToExecutionChanges { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.BlobKzgCommitments) > 0 { + for _, s := range m.BlobKzgCommitments { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.ExecutionRequests != nil { + l = m.ExecutionRequests.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyFulu) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayload != nil { + l = m.ExecutionPayload.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlsToExecutionChanges) > 0 { + for _, e := range m.BlsToExecutionChanges { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.BlobKzgCommitments) > 0 { + for _, s := range m.BlobKzgCommitments { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.ExecutionRequests != nil { + l = m.ExecutionRequests.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlindedBeaconBlockBodyCapella) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayloadHeader != nil { + l = m.ExecutionPayloadHeader.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlsToExecutionChanges) > 0 { + for _, e := range m.BlsToExecutionChanges { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *BlindedBeaconBlockBodyCapellaV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionPayloadHeader != nil { + l = m.ExecutionPayloadHeader.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlsToExecutionChanges) > 0 { + for _, e := range m.BlsToExecutionChanges { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyAltair) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlockBodyAltairV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.RandaoReveal) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Eth1Data != nil { + l = m.Eth1Data.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Graffiti) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ProposerSlashings) > 0 { + for _, e := range m.ProposerSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.AttesterSlashings) > 0 { + for _, e := range m.AttesterSlashings { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Attestations) > 0 { + for _, e := range m.Attestations { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Deposits) > 0 { + for _, e := range m.Deposits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.VoluntaryExits) > 0 { + for _, e := range m.VoluntaryExits { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.SyncAggregate != nil { + l = m.SyncAggregate.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBeaconBlockBellatrix) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockBellatrix: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockBellatrix: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockBellatrix{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockBellatrixV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockBellatrixV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockBellatrixV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockBellatrixV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockCapella) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockCapella: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockCapella: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockCapella{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockCapellaV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockCapellaV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockCapellaV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockCapellaV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockDeneb) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockDeneb: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockDeneb: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockDeneb{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockElectra) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockElectra: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockElectra: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockElectra{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockFulu) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockFulu: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockFulu: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockFulu{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBlindedBeaconBlockBellatrix) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBlindedBeaconBlockBellatrix: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBlindedBeaconBlockBellatrix: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BlindedBeaconBlockBellatrix{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBlindedBeaconBlockBellatrixV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBlindedBeaconBlockBellatrixV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBlindedBeaconBlockBellatrixV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BlindedBeaconBlockBellatrixV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBlindedBeaconBlockCapella) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBlindedBeaconBlockCapella: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBlindedBeaconBlockCapella: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BlindedBeaconBlockCapella{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBlindedBeaconBlockCapellaV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBlindedBeaconBlockCapellaV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBlindedBeaconBlockCapellaV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BlindedBeaconBlockCapellaV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockAltair) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockAltair: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockAltair: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockAltair{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBeaconBlockAltairV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBeaconBlockAltairV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBeaconBlockAltairV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BeaconBlockAltairV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBellatrix) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBellatrix: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBellatrix: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + m.ProposerIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProposerIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyBellatrix{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBellatrixV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBellatrixV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBellatrixV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyBellatrixV2{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlindedBeaconBlockBellatrix) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlindedBeaconBlockBellatrix: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlindedBeaconBlockBellatrix: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + m.ProposerIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProposerIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BlindedBeaconBlockBodyBellatrix{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlindedBeaconBlockBellatrixV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlindedBeaconBlockBellatrixV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlindedBeaconBlockBellatrixV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BlindedBeaconBlockBodyBellatrixV2{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockCapella) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockCapella: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockCapella: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + m.ProposerIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProposerIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyCapella{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockCapellaV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockCapellaV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockCapellaV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyCapellaV2{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockDeneb) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockDeneb: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockDeneb: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyDeneb{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockElectra) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockElectra: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockElectra: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyElectra{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockFulu) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockFulu: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockFulu: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyFulu{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlindedBeaconBlockCapella) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlindedBeaconBlockCapella: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlindedBeaconBlockCapella: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + m.ProposerIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProposerIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BlindedBeaconBlockBodyCapella{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlindedBeaconBlockCapellaV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlindedBeaconBlockCapellaV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlindedBeaconBlockCapellaV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BlindedBeaconBlockBodyCapellaV2{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockAltair) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockAltair: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockAltair: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + m.Slot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Slot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + m.ProposerIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProposerIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyAltair{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockAltairV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockAltairV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockAltairV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Body == nil { + m.Body = &BeaconBlockBodyAltairV2{} + } + if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyBellatrix: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyBellatrix: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayload == nil { + m.ExecutionPayload = &v1.ExecutionPayload{} + } + if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyBellatrixV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyBellatrixV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayload == nil { + m.ExecutionPayload = &v1.ExecutionPayloadV2{} + } + if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlindedBeaconBlockBodyBellatrix: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlindedBeaconBlockBodyBellatrix: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayloadHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayloadHeader == nil { + m.ExecutionPayloadHeader = &v1.ExecutionPayloadHeader{} + } + if err := m.ExecutionPayloadHeader.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlindedBeaconBlockBodyBellatrixV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlindedBeaconBlockBodyBellatrixV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayloadHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayloadHeader == nil { + m.ExecutionPayloadHeader = &v1.ExecutionPayloadHeaderV2{} + } + if err := m.ExecutionPayloadHeader.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyCapella: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyCapella: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayload == nil { + m.ExecutionPayload = &v1.ExecutionPayloadCapella{} + } + if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlsToExecutionChanges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyCapellaV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyCapellaV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayload == nil { + m.ExecutionPayload = &v1.ExecutionPayloadCapellaV2{} + } + if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlsToExecutionChanges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyDeneb: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyDeneb: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayload == nil { + m.ExecutionPayload = &v1.ExecutionPayloadDeneb{} + } + if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlsToExecutionChanges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobKzgCommitments", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobKzgCommitments = append(m.BlobKzgCommitments, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyElectra: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyElectra: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayload == nil { + m.ExecutionPayload = &v1.ExecutionPayloadElectra{} + } + if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlsToExecutionChanges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobKzgCommitments", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobKzgCommitments = append(m.BlobKzgCommitments, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionRequests", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionRequests == nil { + m.ExecutionRequests = &v1.ElectraExecutionRequests{} + } + if err := m.ExecutionRequests.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyFulu: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyFulu: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayload == nil { + m.ExecutionPayload = &v1.ExecutionPayloadElectra{} + } + if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlsToExecutionChanges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobKzgCommitments", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobKzgCommitments = append(m.BlobKzgCommitments, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionRequests", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionRequests == nil { + m.ExecutionRequests = &v1.ElectraExecutionRequests{} + } + if err := m.ExecutionRequests.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlindedBeaconBlockBodyCapella: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlindedBeaconBlockBodyCapella: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayloadHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayloadHeader == nil { + m.ExecutionPayloadHeader = &v1.ExecutionPayloadHeaderCapella{} + } + if err := m.ExecutionPayloadHeader.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlsToExecutionChanges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlindedBeaconBlockBodyCapellaV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlindedBeaconBlockBodyCapellaV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionPayloadHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionPayloadHeader == nil { + m.ExecutionPayloadHeader = &v1.ExecutionPayloadHeaderCapellaV2{} + } + if err := m.ExecutionPayloadHeader.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlsToExecutionChanges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyAltair) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyAltair: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyAltair: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BeaconBlockBodyAltairV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlockBodyAltairV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlockBodyAltairV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RandaoReveal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RandaoReveal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Eth1Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Eth1Data == nil { + m.Eth1Data = &v1.Eth1Data{} + } + if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graffiti", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graffiti = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttesterSlashings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attestations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Attestations = append(m.Attestations, &v1.Attestation{}) + if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Deposits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Deposits = append(m.Deposits, &v1.Deposit{}) + if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoluntaryExits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncAggregate == nil { + m.SyncAggregate = &v1.SyncAggregate{} + } + if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v2/events_vtproto.pb.go b/pkg/proto/eth/v2/events_vtproto.pb.go new file mode 100644 index 00000000..1bf065bc --- /dev/null +++ b/pkg/proto/eth/v2/events_vtproto.pb.go @@ -0,0 +1,1381 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v2/events.proto + +package v2 + +import ( + fmt "fmt" + v1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + io "io" +) + +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) +) + +func (m *EventBlock) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if vtmsg, ok := m.Message.(interface { + MarshalToSizedBufferVT([]byte) (int, error) + }); ok { + size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + } + if m.Version != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Version)) + i-- + dAtA[i] = 0x30 + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x2a + } + return len(dAtA) - i, nil +} + +func (m *EventBlock_Phase0Block) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock_Phase0Block) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Phase0Block != nil { + size, err := m.Phase0Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} +func (m *EventBlock_AltairBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock_AltairBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.AltairBlock != nil { + size, err := m.AltairBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} +func (m *EventBlock_BellatrixBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock_BellatrixBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.BellatrixBlock != nil { + size, err := m.BellatrixBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *EventBlock_CapellaBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock_CapellaBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.CapellaBlock != nil { + size, err := m.CapellaBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + return len(dAtA) - i, nil +} +func (m *EventBlock_DenebBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock_DenebBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.DenebBlock != nil { + size, err := m.DenebBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + return len(dAtA) - i, nil +} +func (m *EventBlock_ElectraBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock_ElectraBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ElectraBlock != nil { + size, err := m.ElectraBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + return len(dAtA) - i, nil +} +func (m *EventBlock_FuluBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlock_FuluBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.FuluBlock != nil { + size, err := m.FuluBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + return len(dAtA) - i, nil +} +func (m *EventBlockV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EventBlockV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if vtmsg, ok := m.Message.(interface { + MarshalToSizedBufferVT([]byte) (int, error) + }); ok { + size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + } + if m.Version != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Version)) + i-- + dAtA[i] = 0x30 + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x2a + } + return len(dAtA) - i, nil +} + +func (m *EventBlockV2_Phase0Block) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2_Phase0Block) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Phase0Block != nil { + size, err := m.Phase0Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} +func (m *EventBlockV2_AltairBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2_AltairBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.AltairBlock != nil { + size, err := m.AltairBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} +func (m *EventBlockV2_BellatrixBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2_BellatrixBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.BellatrixBlock != nil { + size, err := m.BellatrixBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *EventBlockV2_CapellaBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2_CapellaBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.CapellaBlock != nil { + size, err := m.CapellaBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + return len(dAtA) - i, nil +} +func (m *EventBlockV2_DenebBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2_DenebBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.DenebBlock != nil { + size, err := m.DenebBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + return len(dAtA) - i, nil +} +func (m *EventBlockV2_ElectraBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2_ElectraBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ElectraBlock != nil { + size, err := m.ElectraBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + return len(dAtA) - i, nil +} +func (m *EventBlockV2_FuluBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EventBlockV2_FuluBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.FuluBlock != nil { + size, err := m.FuluBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + return len(dAtA) - i, nil +} +func (m *EventBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if vtmsg, ok := m.Message.(interface{ SizeVT() int }); ok { + n += vtmsg.SizeVT() + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Version != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Version)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventBlock_Phase0Block) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Phase0Block != nil { + l = m.Phase0Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlock_AltairBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AltairBlock != nil { + l = m.AltairBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlock_BellatrixBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BellatrixBlock != nil { + l = m.BellatrixBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlock_CapellaBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CapellaBlock != nil { + l = m.CapellaBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlock_DenebBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.DenebBlock != nil { + l = m.DenebBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlock_ElectraBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ElectraBlock != nil { + l = m.ElectraBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlock_FuluBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.FuluBlock != nil { + l = m.FuluBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlockV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if vtmsg, ok := m.Message.(interface{ SizeVT() int }); ok { + n += vtmsg.SizeVT() + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Version != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Version)) + } + n += len(m.unknownFields) + return n +} + +func (m *EventBlockV2_Phase0Block) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Phase0Block != nil { + l = m.Phase0Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlockV2_AltairBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AltairBlock != nil { + l = m.AltairBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlockV2_BellatrixBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BellatrixBlock != nil { + l = m.BellatrixBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlockV2_CapellaBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CapellaBlock != nil { + l = m.CapellaBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlockV2_DenebBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.DenebBlock != nil { + l = m.DenebBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlockV2_ElectraBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ElectraBlock != nil { + l = m.ElectraBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlockV2_FuluBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.FuluBlock != nil { + l = m.FuluBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *EventBlock) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventBlock: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventBlock: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Phase0Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlock_Phase0Block); ok { + if err := oneof.Phase0Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.BeaconBlock{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlock_Phase0Block{Phase0Block: v} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AltairBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlock_AltairBlock); ok { + if err := oneof.AltairBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockAltair{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlock_AltairBlock{AltairBlock: v} + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BellatrixBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlock_BellatrixBlock); ok { + if err := oneof.BellatrixBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockBellatrix{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlock_BellatrixBlock{BellatrixBlock: v} + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CapellaBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlock_CapellaBlock); ok { + if err := oneof.CapellaBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockCapella{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlock_CapellaBlock{CapellaBlock: v} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + m.Version = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Version |= BlockVersion(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DenebBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlock_DenebBlock); ok { + if err := oneof.DenebBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockDeneb{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlock_DenebBlock{DenebBlock: v} + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ElectraBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlock_ElectraBlock); ok { + if err := oneof.ElectraBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockElectra{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlock_ElectraBlock{ElectraBlock: v} + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FuluBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlock_FuluBlock); ok { + if err := oneof.FuluBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockFulu{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlock_FuluBlock{FuluBlock: v} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EventBlockV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EventBlockV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Phase0Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlockV2_Phase0Block); ok { + if err := oneof.Phase0Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.BeaconBlockV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlockV2_Phase0Block{Phase0Block: v} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AltairBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlockV2_AltairBlock); ok { + if err := oneof.AltairBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockAltairV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlockV2_AltairBlock{AltairBlock: v} + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BellatrixBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlockV2_BellatrixBlock); ok { + if err := oneof.BellatrixBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockBellatrixV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlockV2_BellatrixBlock{BellatrixBlock: v} + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CapellaBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlockV2_CapellaBlock); ok { + if err := oneof.CapellaBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockCapellaV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlockV2_CapellaBlock{CapellaBlock: v} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + m.Version = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Version |= BlockVersion(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DenebBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlockV2_DenebBlock); ok { + if err := oneof.DenebBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockDeneb{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlockV2_DenebBlock{DenebBlock: v} + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ElectraBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlockV2_ElectraBlock); ok { + if err := oneof.ElectraBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockElectra{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlockV2_ElectraBlock{ElectraBlock: v} + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FuluBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Message.(*EventBlockV2_FuluBlock); ok { + if err := oneof.FuluBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &BeaconBlockFulu{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Message = &EventBlockV2_FuluBlock{FuluBlock: v} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/eth/v2/withdrawals_vtproto.pb.go b/pkg/proto/eth/v2/withdrawals_vtproto.pb.go new file mode 100644 index 00000000..7b125578 --- /dev/null +++ b/pkg/proto/eth/v2/withdrawals_vtproto.pb.go @@ -0,0 +1,979 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/eth/v2/withdrawals.proto + +package v2 + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *BLSToExecutionChange) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BLSToExecutionChange) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BLSToExecutionChange) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ToExecutionAddress) > 0 { + i -= len(m.ToExecutionAddress) + copy(dAtA[i:], m.ToExecutionAddress) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ToExecutionAddress))) + i-- + dAtA[i] = 0x1a + } + if len(m.FromBlsPubkey) > 0 { + i -= len(m.FromBlsPubkey) + copy(dAtA[i:], m.FromBlsPubkey) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FromBlsPubkey))) + i-- + dAtA[i] = 0x12 + } + if m.ValidatorIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ValidatorIndex)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *BLSToExecutionChangeV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BLSToExecutionChangeV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BLSToExecutionChangeV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ToExecutionAddress) > 0 { + i -= len(m.ToExecutionAddress) + copy(dAtA[i:], m.ToExecutionAddress) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ToExecutionAddress))) + i-- + dAtA[i] = 0x1a + } + if len(m.FromBlsPubkey) > 0 { + i -= len(m.FromBlsPubkey) + copy(dAtA[i:], m.FromBlsPubkey) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FromBlsPubkey))) + i-- + dAtA[i] = 0x12 + } + if m.ValidatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBLSToExecutionChange) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBLSToExecutionChange) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBLSToExecutionChange) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SignedBLSToExecutionChangeV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SignedBLSToExecutionChangeV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SignedBLSToExecutionChangeV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SubmitBLSToExecutionChangesRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SubmitBLSToExecutionChangesRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SubmitBLSToExecutionChangesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Changes) > 0 { + for iNdEx := len(m.Changes) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Changes[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *BLSToExecutionChange) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ValidatorIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ValidatorIndex)) + } + l = len(m.FromBlsPubkey) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ToExecutionAddress) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BLSToExecutionChangeV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ValidatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ValidatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FromBlsPubkey) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ToExecutionAddress) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBLSToExecutionChange) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SignedBLSToExecutionChangeV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SubmitBLSToExecutionChangesRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Changes) > 0 { + for _, e := range m.Changes { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *BLSToExecutionChange) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BLSToExecutionChange: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BLSToExecutionChange: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + m.ValidatorIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ValidatorIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FromBlsPubkey", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FromBlsPubkey = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ToExecutionAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ToExecutionAddress = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BLSToExecutionChangeV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BLSToExecutionChangeV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BLSToExecutionChangeV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidatorIndex == nil { + m.ValidatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FromBlsPubkey", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FromBlsPubkey = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ToExecutionAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ToExecutionAddress = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBLSToExecutionChange) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBLSToExecutionChange: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBLSToExecutionChange: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BLSToExecutionChange{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SignedBLSToExecutionChangeV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SignedBLSToExecutionChangeV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SignedBLSToExecutionChangeV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &BLSToExecutionChangeV2{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SubmitBLSToExecutionChangesRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SubmitBLSToExecutionChangesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SubmitBLSToExecutionChangesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Changes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Changes = append(m.Changes, &SignedBLSToExecutionChange{}) + if err := m.Changes[len(m.Changes)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/libp2p/eth_vtproto.pb.go b/pkg/proto/libp2p/eth_vtproto.pb.go new file mode 100644 index 00000000..c674474d --- /dev/null +++ b/pkg/proto/libp2p/eth_vtproto.pb.go @@ -0,0 +1,710 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/libp2p/eth.proto + +package libp2p + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *Status) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Status) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Status) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.EarliestAvailableSlot != nil { + size, err := (*wrapperspb.UInt64Value)(m.EarliestAvailableSlot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.HeadSlot != nil { + size, err := (*wrapperspb.UInt64Value)(m.HeadSlot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.HeadRoot != nil { + size, err := (*wrapperspb.StringValue)(m.HeadRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.FinalizedEpoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.FinalizedEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.FinalizedRoot != nil { + size, err := (*wrapperspb.StringValue)(m.FinalizedRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.ForkDigest != nil { + size, err := (*wrapperspb.StringValue)(m.ForkDigest).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Metadata) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Metadata) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Metadata) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.CustodyGroupCount != nil { + size, err := (*wrapperspb.UInt64Value)(m.CustodyGroupCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Syncnets != nil { + size, err := (*wrapperspb.StringValue)(m.Syncnets).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Attnets != nil { + size, err := (*wrapperspb.StringValue)(m.Attnets).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.SeqNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.SeqNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Status) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ForkDigest != nil { + l = (*wrapperspb.StringValue)(m.ForkDigest).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedRoot != nil { + l = (*wrapperspb.StringValue)(m.FinalizedRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedEpoch != nil { + l = (*wrapperspb.UInt64Value)(m.FinalizedEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HeadRoot != nil { + l = (*wrapperspb.StringValue)(m.HeadRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HeadSlot != nil { + l = (*wrapperspb.UInt64Value)(m.HeadSlot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.EarliestAvailableSlot != nil { + l = (*wrapperspb.UInt64Value)(m.EarliestAvailableSlot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Metadata) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SeqNumber != nil { + l = (*wrapperspb.UInt64Value)(m.SeqNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Attnets != nil { + l = (*wrapperspb.StringValue)(m.Attnets).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Syncnets != nil { + l = (*wrapperspb.StringValue)(m.Syncnets).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.CustodyGroupCount != nil { + l = (*wrapperspb.UInt64Value)(m.CustodyGroupCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Status) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Status: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Status: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkDigest", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ForkDigest == nil { + m.ForkDigest = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ForkDigest).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedRoot == nil { + m.FinalizedRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.FinalizedRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedEpoch == nil { + m.FinalizedEpoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.FinalizedEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HeadRoot == nil { + m.HeadRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.HeadRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HeadSlot == nil { + m.HeadSlot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.HeadSlot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EarliestAvailableSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EarliestAvailableSlot == nil { + m.EarliestAvailableSlot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.EarliestAvailableSlot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Metadata) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Metadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Metadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SeqNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SeqNumber == nil { + m.SeqNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SeqNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Attnets", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Attnets == nil { + m.Attnets = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Attnets).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Syncnets", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Syncnets == nil { + m.Syncnets = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Syncnets).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CustodyGroupCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CustodyGroupCount == nil { + m.CustodyGroupCount = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.CustodyGroupCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go b/pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go new file mode 100644 index 00000000..36a7f0a1 --- /dev/null +++ b/pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go @@ -0,0 +1,1104 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/libp2p/gossipsub/eth.proto + +package gossipsub + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *BeaconBlock) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Block != nil { + size, err := (*wrapperspb.StringValue)(m.Block).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlobSidecar) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BlockRoot != nil { + size, err := (*wrapperspb.StringValue)(m.BlockRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.ParentRoot != nil { + size, err := (*wrapperspb.StringValue)(m.ParentRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.StateRoot != nil { + size, err := (*wrapperspb.StringValue)(m.StateRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DataColumnSidecar) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DataColumnSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DataColumnSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BlockRoot != nil { + size, err := (*wrapperspb.StringValue)(m.BlockRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.KzgCommitmentsCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.KzgCommitmentsCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.ParentRoot != nil { + size, err := (*wrapperspb.StringValue)(m.ParentRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.StateRoot != nil { + size, err := (*wrapperspb.StringValue)(m.StateRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = (*wrapperspb.StringValue)(m.Block).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StateRoot != nil { + l = (*wrapperspb.StringValue)(m.StateRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ParentRoot != nil { + l = (*wrapperspb.StringValue)(m.ParentRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockRoot != nil { + l = (*wrapperspb.StringValue)(m.BlockRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DataColumnSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StateRoot != nil { + l = (*wrapperspb.StringValue)(m.StateRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ParentRoot != nil { + l = (*wrapperspb.StringValue)(m.ParentRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.KzgCommitmentsCount != nil { + l = (*wrapperspb.UInt32Value)(m.KzgCommitmentsCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockRoot != nil { + l = (*wrapperspb.StringValue)(m.BlockRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BeaconBlock) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BeaconBlock: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BeaconBlock: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Block).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlobSidecar) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlobSidecar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlobSidecar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StateRoot == nil { + m.StateRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.StateRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ParentRoot == nil { + m.ParentRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ParentRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockRoot == nil { + m.BlockRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.BlockRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DataColumnSidecar) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DataColumnSidecar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DataColumnSidecar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StateRoot == nil { + m.StateRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.StateRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ParentRoot == nil { + m.ParentRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ParentRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KzgCommitmentsCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.KzgCommitmentsCount == nil { + m.KzgCommitmentsCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.KzgCommitmentsCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockRoot == nil { + m.BlockRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.BlockRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/libp2p/peer_vtproto.pb.go b/pkg/proto/libp2p/peer_vtproto.pb.go new file mode 100644 index 00000000..d021498e --- /dev/null +++ b/pkg/proto/libp2p/peer_vtproto.pb.go @@ -0,0 +1,435 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/libp2p/peer.proto + +package libp2p + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *Peer) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Peer) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Peer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Latency != nil { + size, err := (*wrapperspb.UInt64Value)(m.Latency).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.Protocols) > 0 { + for iNdEx := len(m.Protocols) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Protocols[iNdEx]) + copy(dAtA[i:], m.Protocols[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Protocols[iNdEx]))) + i-- + dAtA[i] = 0x32 + } + } + if len(m.ProtocolVersion) > 0 { + i -= len(m.ProtocolVersion) + copy(dAtA[i:], m.ProtocolVersion) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ProtocolVersion))) + i-- + dAtA[i] = 0x2a + } + if len(m.UserAgent) > 0 { + i -= len(m.UserAgent) + copy(dAtA[i:], m.UserAgent) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.UserAgent))) + i-- + dAtA[i] = 0x22 + } + if len(m.Ip) > 0 { + i -= len(m.Ip) + copy(dAtA[i:], m.Ip) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Ip))) + i-- + dAtA[i] = 0x1a + } + if m.Port != nil { + size, err := (*wrapperspb.UInt32Value)(m.Port).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.Id) > 0 { + i -= len(m.Id) + copy(dAtA[i:], m.Id) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Id))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Peer) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Id) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Port != nil { + l = (*wrapperspb.UInt32Value)(m.Port).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Ip) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.UserAgent) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ProtocolVersion) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Protocols) > 0 { + for _, s := range m.Protocols { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.Latency != nil { + l = (*wrapperspb.UInt64Value)(m.Latency).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Peer) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Peer: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Peer: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Id = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Port", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Port == nil { + m.Port = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Port).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Ip", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Ip = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UserAgent", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.UserAgent = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProtocolVersion", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProtocolVersion = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Protocols", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Protocols = append(m.Protocols, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Latency", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Latency == nil { + m.Latency = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Latency).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/libp2p/trace_vtproto.pb.go b/pkg/proto/libp2p/trace_vtproto.pb.go new file mode 100644 index 00000000..121d8f87 --- /dev/null +++ b/pkg/proto/libp2p/trace_vtproto.pb.go @@ -0,0 +1,11367 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/libp2p/trace.proto + +package libp2p + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + timestamppb "github.com/planetscale/vtprotobuf/types/known/timestamppb" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *AddPeer) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AddPeer) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AddPeer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Protocol != nil { + size, err := (*wrapperspb.StringValue)(m.Protocol).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RemovePeer) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RemovePeer) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RemovePeer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RecvRPC) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RecvRPC) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RecvRPC) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Meta != nil { + size, err := m.Meta.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SendRPC) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SendRPC) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SendRPC) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Meta != nil { + size, err := m.Meta.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DropRPC) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DropRPC) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DropRPC) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Meta != nil { + size, err := m.Meta.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Join) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Join) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Join) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Leave) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Leave) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Leave) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *Graft) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Graft) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Graft) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Prune) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Prune) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Prune) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *PublishMessage) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *PublishMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *PublishMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.MsgId != nil { + size, err := (*wrapperspb.StringValue)(m.MsgId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RejectMessage) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RejectMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RejectMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SeqNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.SeqNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.MsgSize != nil { + size, err := (*wrapperspb.UInt32Value)(m.MsgSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Local != nil { + size, err := (*wrapperspb.BoolValue)(m.Local).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Reason != nil { + size, err := (*wrapperspb.StringValue)(m.Reason).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.MsgId != nil { + size, err := (*wrapperspb.StringValue)(m.MsgId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DuplicateMessage) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DuplicateMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DuplicateMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SeqNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.SeqNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.MsgSize != nil { + size, err := (*wrapperspb.UInt32Value)(m.MsgSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Local != nil { + size, err := (*wrapperspb.BoolValue)(m.Local).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.MsgId != nil { + size, err := (*wrapperspb.StringValue)(m.MsgId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DeliverMessage) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DeliverMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DeliverMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SeqNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.SeqNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.MsgSize != nil { + size, err := (*wrapperspb.UInt32Value)(m.MsgSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Local != nil { + size, err := (*wrapperspb.BoolValue)(m.Local).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.MsgId != nil { + size, err := (*wrapperspb.StringValue)(m.MsgId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *TraceEventMetadata) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TraceEventMetadata) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *TraceEventMetadata) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Connected) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Connected) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Connected) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Limited != nil { + size, err := (*wrapperspb.BoolValue)(m.Limited).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.Transient != nil { + size, err := (*wrapperspb.BoolValue)(m.Transient).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Opened != nil { + size, err := (*timestamppb.Timestamp)(m.Opened).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Direction != nil { + size, err := (*wrapperspb.StringValue)(m.Direction).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.AgentVersion != nil { + size, err := (*wrapperspb.StringValue)(m.AgentVersion).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.RemoteMaddrs != nil { + size, err := (*wrapperspb.StringValue)(m.RemoteMaddrs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RemotePeer != nil { + size, err := (*wrapperspb.StringValue)(m.RemotePeer).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Disconnected) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Disconnected) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Disconnected) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Limited != nil { + size, err := (*wrapperspb.BoolValue)(m.Limited).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.Transient != nil { + size, err := (*wrapperspb.BoolValue)(m.Transient).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Opened != nil { + size, err := (*timestamppb.Timestamp)(m.Opened).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Direction != nil { + size, err := (*wrapperspb.StringValue)(m.Direction).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.AgentVersion != nil { + size, err := (*wrapperspb.StringValue)(m.AgentVersion).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.RemoteMaddrs != nil { + size, err := (*wrapperspb.StringValue)(m.RemoteMaddrs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RemotePeer != nil { + size, err := (*wrapperspb.StringValue)(m.RemotePeer).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *HandleStatus) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *HandleStatus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *HandleStatus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Direction != nil { + size, err := (*wrapperspb.StringValue)(m.Direction).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.Response != nil { + size, err := m.Response.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Request != nil { + size, err := m.Request.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Latency != nil { + size, err := (*wrapperspb.FloatValue)(m.Latency).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Error != nil { + size, err := (*wrapperspb.StringValue)(m.Error).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.ProtocolId != nil { + size, err := (*wrapperspb.StringValue)(m.ProtocolId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *HandleMetadata) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *HandleMetadata) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *HandleMetadata) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Direction != nil { + size, err := (*wrapperspb.StringValue)(m.Direction).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Latency != nil { + size, err := (*wrapperspb.FloatValue)(m.Latency).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Error != nil { + size, err := (*wrapperspb.StringValue)(m.Error).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.ProtocolId != nil { + size, err := (*wrapperspb.StringValue)(m.ProtocolId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SyntheticHeartbeat) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyntheticHeartbeat) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SyntheticHeartbeat) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ConnectionAgeNs != nil { + size, err := (*wrapperspb.Int64Value)(m.ConnectionAgeNs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if len(m.Protocols) > 0 { + for iNdEx := len(m.Protocols) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Protocols[iNdEx]) + copy(dAtA[i:], m.Protocols[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Protocols[iNdEx]))) + i-- + dAtA[i] = 0x3a + } + } + if m.Direction != nil { + size, err := (*wrapperspb.UInt32Value)(m.Direction).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.AgentVersion != nil { + size, err := (*wrapperspb.StringValue)(m.AgentVersion).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.LatencyMs != nil { + size, err := (*wrapperspb.Int64Value)(m.LatencyMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.RemoteMaddrs != nil { + size, err := (*wrapperspb.StringValue)(m.RemoteMaddrs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.RemotePeer != nil { + size, err := (*wrapperspb.StringValue)(m.RemotePeer).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Timestamp != nil { + size, err := (*timestamppb.Timestamp)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RPCMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RPCMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RPCMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Control != nil { + size, err := m.Control.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.Subscriptions) > 0 { + for iNdEx := len(m.Subscriptions) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Subscriptions[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Messages) > 0 { + for iNdEx := len(m.Messages) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Messages[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *MessageMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *MessageMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *MessageMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.TopicId != nil { + size, err := (*wrapperspb.StringValue)(m.TopicId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *MessageMetaItem) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *MessageMetaItem) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *MessageMetaItem) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ControlIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.ControlIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.TopicId != nil { + size, err := (*wrapperspb.StringValue)(m.TopicId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RootEventId != nil { + size, err := (*wrapperspb.StringValue)(m.RootEventId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SubMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SubMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SubMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.TopicId != nil { + size, err := (*wrapperspb.StringValue)(m.TopicId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Subscribe != nil { + size, err := (*wrapperspb.BoolValue)(m.Subscribe).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SubMetaItem) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SubMetaItem) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SubMetaItem) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ControlIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.ControlIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Subscribe != nil { + size, err := (*wrapperspb.BoolValue)(m.Subscribe).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.TopicId != nil { + size, err := (*wrapperspb.StringValue)(m.TopicId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RootEventId != nil { + size, err := (*wrapperspb.StringValue)(m.RootEventId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ControlMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Idontwant) > 0 { + for iNdEx := len(m.Idontwant) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Idontwant[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.Prune) > 0 { + for iNdEx := len(m.Prune) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Prune[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.Graft) > 0 { + for iNdEx := len(m.Graft) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Graft[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if len(m.Iwant) > 0 { + for iNdEx := len(m.Iwant) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Iwant[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Ihave) > 0 { + for iNdEx := len(m.Ihave) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Ihave[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *ControlIHaveMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlIHaveMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlIHaveMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.MessageIds) > 0 { + for iNdEx := len(m.MessageIds) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.StringValue)(m.MessageIds[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + if m.TopicId != nil { + size, err := (*wrapperspb.StringValue)(m.TopicId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ControlIHaveMetaItem) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlIHaveMetaItem) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlIHaveMetaItem) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ControlIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.ControlIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.MessageIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.MessageIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RootEventId != nil { + size, err := (*wrapperspb.StringValue)(m.RootEventId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ControlIWantMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlIWantMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlIWantMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.MessageIds) > 0 { + for iNdEx := len(m.MessageIds) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.StringValue)(m.MessageIds[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + return len(dAtA) - i, nil +} + +func (m *ControlIWantMetaItem) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlIWantMetaItem) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlIWantMetaItem) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ControlIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.ControlIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.MessageIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.MessageIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RootEventId != nil { + size, err := (*wrapperspb.StringValue)(m.RootEventId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ControlGraftMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlGraftMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlGraftMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.TopicId != nil { + size, err := (*wrapperspb.StringValue)(m.TopicId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ControlGraftMetaItem) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlGraftMetaItem) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlGraftMetaItem) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ControlIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.ControlIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RootEventId != nil { + size, err := (*wrapperspb.StringValue)(m.RootEventId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ControlPruneMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlPruneMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlPruneMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.PeerIds) > 0 { + for iNdEx := len(m.PeerIds) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.StringValue)(m.PeerIds[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + if m.TopicId != nil { + size, err := (*wrapperspb.StringValue)(m.TopicId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ControlPruneMetaItem) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlPruneMetaItem) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlPruneMetaItem) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.PeerIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.PeerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.ControlIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.ControlIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.GraftPeerId != nil { + size, err := (*wrapperspb.StringValue)(m.GraftPeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RootEventId != nil { + size, err := (*wrapperspb.StringValue)(m.RootEventId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ControlIDontWantMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlIDontWantMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlIDontWantMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.MessageIds) > 0 { + for iNdEx := len(m.MessageIds) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.StringValue)(m.MessageIds[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *ControlIDontWantMetaItem) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ControlIDontWantMetaItem) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ControlIDontWantMetaItem) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ControlIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.ControlIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.MessageIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.MessageIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RootEventId != nil { + size, err := (*wrapperspb.StringValue)(m.RootEventId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DataColumnCustodyProbe) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DataColumnCustodyProbe) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DataColumnCustodyProbe) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Error != nil { + size, err := (*wrapperspb.StringValue)(m.Error).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if m.ResponseTimeMs != nil { + size, err := (*wrapperspb.Int64Value)(m.ResponseTimeMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + if m.Result != nil { + size, err := (*wrapperspb.StringValue)(m.Result).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x72 + } + if m.BeaconBlockRoot != nil { + size, err := (*wrapperspb.StringValue)(m.BeaconBlockRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if m.ColumnRowsCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.ColumnRowsCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.ColumnIndex != nil { + size, err := (*wrapperspb.UInt32Value)(m.ColumnIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.WallclockRequestEpochStartDateTime != nil { + size, err := (*wrapperspb.UInt64Value)(m.WallclockRequestEpochStartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.WallclockRequestEpoch != nil { + size, err := (*wrapperspb.UInt32Value)(m.WallclockRequestEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.WallclockRequestSlotStartDateTime != nil { + size, err := (*wrapperspb.UInt64Value)(m.WallclockRequestSlotStartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.WallclockRequestSlot != nil { + size, err := (*wrapperspb.UInt32Value)(m.WallclockRequestSlot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.EpochStartDateTime != nil { + size, err := (*wrapperspb.UInt64Value)(m.EpochStartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Epoch != nil { + size, err := (*wrapperspb.UInt32Value)(m.Epoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.SlotStartDateTime != nil { + size, err := (*wrapperspb.UInt64Value)(m.SlotStartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt32Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.JobStartTimestamp != nil { + size, err := (*timestamppb.Timestamp)(m.JobStartTimestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Identify) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Identify) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Identify) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.RemoteMultiaddr != nil { + size, err := (*wrapperspb.StringValue)(m.RemoteMultiaddr).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if m.Direction != nil { + size, err := (*wrapperspb.StringValue)(m.Direction).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.Muxer != nil { + size, err := (*wrapperspb.StringValue)(m.Muxer).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.Security != nil { + size, err := (*wrapperspb.StringValue)(m.Security).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.Transport != nil { + size, err := (*wrapperspb.StringValue)(m.Transport).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.ObservedAddr != nil { + size, err := (*wrapperspb.StringValue)(m.ObservedAddr).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if len(m.ListenAddrs) > 0 { + for iNdEx := len(m.ListenAddrs) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ListenAddrs[iNdEx]) + copy(dAtA[i:], m.ListenAddrs[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ListenAddrs[iNdEx]))) + i-- + dAtA[i] = 0x3a + } + } + if len(m.Protocols) > 0 { + for iNdEx := len(m.Protocols) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Protocols[iNdEx]) + copy(dAtA[i:], m.Protocols[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Protocols[iNdEx]))) + i-- + dAtA[i] = 0x32 + } + } + if m.ProtocolVersion != nil { + size, err := (*wrapperspb.StringValue)(m.ProtocolVersion).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.AgentVersion != nil { + size, err := (*wrapperspb.StringValue)(m.AgentVersion).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Error != nil { + size, err := (*wrapperspb.StringValue)(m.Error).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Success != nil { + size, err := (*wrapperspb.BoolValue)(m.Success).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RemotePeer != nil { + size, err := (*wrapperspb.StringValue)(m.RemotePeer).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *AddPeer) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Protocol != nil { + l = (*wrapperspb.StringValue)(m.Protocol).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *RemovePeer) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *RecvRPC) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Meta != nil { + l = m.Meta.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SendRPC) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Meta != nil { + l = m.Meta.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DropRPC) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Meta != nil { + l = m.Meta.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Join) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Leave) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Graft) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Prune) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *PublishMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MsgId != nil { + l = (*wrapperspb.StringValue)(m.MsgId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *RejectMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MsgId != nil { + l = (*wrapperspb.StringValue)(m.MsgId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Reason != nil { + l = (*wrapperspb.StringValue)(m.Reason).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Local != nil { + l = (*wrapperspb.BoolValue)(m.Local).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MsgSize != nil { + l = (*wrapperspb.UInt32Value)(m.MsgSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SeqNumber != nil { + l = (*wrapperspb.UInt64Value)(m.SeqNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DuplicateMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MsgId != nil { + l = (*wrapperspb.StringValue)(m.MsgId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Local != nil { + l = (*wrapperspb.BoolValue)(m.Local).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MsgSize != nil { + l = (*wrapperspb.UInt32Value)(m.MsgSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SeqNumber != nil { + l = (*wrapperspb.UInt64Value)(m.SeqNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DeliverMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MsgId != nil { + l = (*wrapperspb.StringValue)(m.MsgId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Local != nil { + l = (*wrapperspb.BoolValue)(m.Local).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MsgSize != nil { + l = (*wrapperspb.UInt32Value)(m.MsgSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SeqNumber != nil { + l = (*wrapperspb.UInt64Value)(m.SeqNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *TraceEventMetadata) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Connected) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RemotePeer != nil { + l = (*wrapperspb.StringValue)(m.RemotePeer).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RemoteMaddrs != nil { + l = (*wrapperspb.StringValue)(m.RemoteMaddrs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AgentVersion != nil { + l = (*wrapperspb.StringValue)(m.AgentVersion).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Direction != nil { + l = (*wrapperspb.StringValue)(m.Direction).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Opened != nil { + l = (*timestamppb.Timestamp)(m.Opened).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Transient != nil { + l = (*wrapperspb.BoolValue)(m.Transient).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Limited != nil { + l = (*wrapperspb.BoolValue)(m.Limited).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Disconnected) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RemotePeer != nil { + l = (*wrapperspb.StringValue)(m.RemotePeer).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RemoteMaddrs != nil { + l = (*wrapperspb.StringValue)(m.RemoteMaddrs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AgentVersion != nil { + l = (*wrapperspb.StringValue)(m.AgentVersion).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Direction != nil { + l = (*wrapperspb.StringValue)(m.Direction).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Opened != nil { + l = (*timestamppb.Timestamp)(m.Opened).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Transient != nil { + l = (*wrapperspb.BoolValue)(m.Transient).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Limited != nil { + l = (*wrapperspb.BoolValue)(m.Limited).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *HandleStatus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProtocolId != nil { + l = (*wrapperspb.StringValue)(m.ProtocolId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Error != nil { + l = (*wrapperspb.StringValue)(m.Error).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Latency != nil { + l = (*wrapperspb.FloatValue)(m.Latency).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Request != nil { + l = m.Request.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Response != nil { + l = m.Response.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Direction != nil { + l = (*wrapperspb.StringValue)(m.Direction).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *HandleMetadata) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProtocolId != nil { + l = (*wrapperspb.StringValue)(m.ProtocolId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Error != nil { + l = (*wrapperspb.StringValue)(m.Error).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Latency != nil { + l = (*wrapperspb.FloatValue)(m.Latency).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Direction != nil { + l = (*wrapperspb.StringValue)(m.Direction).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SyntheticHeartbeat) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Timestamp != nil { + l = (*timestamppb.Timestamp)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RemotePeer != nil { + l = (*wrapperspb.StringValue)(m.RemotePeer).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RemoteMaddrs != nil { + l = (*wrapperspb.StringValue)(m.RemoteMaddrs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.LatencyMs != nil { + l = (*wrapperspb.Int64Value)(m.LatencyMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AgentVersion != nil { + l = (*wrapperspb.StringValue)(m.AgentVersion).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Direction != nil { + l = (*wrapperspb.UInt32Value)(m.Direction).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Protocols) > 0 { + for _, s := range m.Protocols { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.ConnectionAgeNs != nil { + l = (*wrapperspb.Int64Value)(m.ConnectionAgeNs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *RPCMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Messages) > 0 { + for _, e := range m.Messages { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Subscriptions) > 0 { + for _, e := range m.Subscriptions { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.Control != nil { + l = m.Control.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *MessageMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TopicId != nil { + l = (*wrapperspb.StringValue)(m.TopicId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *MessageMetaItem) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RootEventId != nil { + l = (*wrapperspb.StringValue)(m.RootEventId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TopicId != nil { + l = (*wrapperspb.StringValue)(m.TopicId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ControlIndex != nil { + l = (*wrapperspb.UInt32Value)(m.ControlIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SubMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Subscribe != nil { + l = (*wrapperspb.BoolValue)(m.Subscribe).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TopicId != nil { + l = (*wrapperspb.StringValue)(m.TopicId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SubMetaItem) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RootEventId != nil { + l = (*wrapperspb.StringValue)(m.RootEventId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TopicId != nil { + l = (*wrapperspb.StringValue)(m.TopicId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Subscribe != nil { + l = (*wrapperspb.BoolValue)(m.Subscribe).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ControlIndex != nil { + l = (*wrapperspb.UInt32Value)(m.ControlIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ControlMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Ihave) > 0 { + for _, e := range m.Ihave { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Iwant) > 0 { + for _, e := range m.Iwant { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Graft) > 0 { + for _, e := range m.Graft { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Prune) > 0 { + for _, e := range m.Prune { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.Idontwant) > 0 { + for _, e := range m.Idontwant { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ControlIHaveMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TopicId != nil { + l = (*wrapperspb.StringValue)(m.TopicId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.MessageIds) > 0 { + for _, e := range m.MessageIds { + l = (*wrapperspb.StringValue)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ControlIHaveMetaItem) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RootEventId != nil { + l = (*wrapperspb.StringValue)(m.RootEventId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageIndex != nil { + l = (*wrapperspb.UInt32Value)(m.MessageIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ControlIndex != nil { + l = (*wrapperspb.UInt32Value)(m.ControlIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ControlIWantMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.MessageIds) > 0 { + for _, e := range m.MessageIds { + l = (*wrapperspb.StringValue)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ControlIWantMetaItem) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RootEventId != nil { + l = (*wrapperspb.StringValue)(m.RootEventId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageIndex != nil { + l = (*wrapperspb.UInt32Value)(m.MessageIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ControlIndex != nil { + l = (*wrapperspb.UInt32Value)(m.ControlIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ControlGraftMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TopicId != nil { + l = (*wrapperspb.StringValue)(m.TopicId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ControlGraftMetaItem) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RootEventId != nil { + l = (*wrapperspb.StringValue)(m.RootEventId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ControlIndex != nil { + l = (*wrapperspb.UInt32Value)(m.ControlIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ControlPruneMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TopicId != nil { + l = (*wrapperspb.StringValue)(m.TopicId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.PeerIds) > 0 { + for _, e := range m.PeerIds { + l = (*wrapperspb.StringValue)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ControlPruneMetaItem) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RootEventId != nil { + l = (*wrapperspb.StringValue)(m.RootEventId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GraftPeerId != nil { + l = (*wrapperspb.StringValue)(m.GraftPeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ControlIndex != nil { + l = (*wrapperspb.UInt32Value)(m.ControlIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerIndex != nil { + l = (*wrapperspb.UInt32Value)(m.PeerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ControlIDontWantMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.MessageIds) > 0 { + for _, e := range m.MessageIds { + l = (*wrapperspb.StringValue)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ControlIDontWantMetaItem) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RootEventId != nil { + l = (*wrapperspb.StringValue)(m.RootEventId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageIndex != nil { + l = (*wrapperspb.UInt32Value)(m.MessageIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ControlIndex != nil { + l = (*wrapperspb.UInt32Value)(m.ControlIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DataColumnCustodyProbe) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.JobStartTimestamp != nil { + l = (*timestamppb.Timestamp)(m.JobStartTimestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt32Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SlotStartDateTime != nil { + l = (*wrapperspb.UInt64Value)(m.SlotStartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = (*wrapperspb.UInt32Value)(m.Epoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.EpochStartDateTime != nil { + l = (*wrapperspb.UInt64Value)(m.EpochStartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockRequestSlot != nil { + l = (*wrapperspb.UInt32Value)(m.WallclockRequestSlot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockRequestSlotStartDateTime != nil { + l = (*wrapperspb.UInt64Value)(m.WallclockRequestSlotStartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockRequestEpoch != nil { + l = (*wrapperspb.UInt32Value)(m.WallclockRequestEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockRequestEpochStartDateTime != nil { + l = (*wrapperspb.UInt64Value)(m.WallclockRequestEpochStartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ColumnIndex != nil { + l = (*wrapperspb.UInt32Value)(m.ColumnIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ColumnRowsCount != nil { + l = (*wrapperspb.UInt32Value)(m.ColumnRowsCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BeaconBlockRoot != nil { + l = (*wrapperspb.StringValue)(m.BeaconBlockRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Result != nil { + l = (*wrapperspb.StringValue)(m.Result).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ResponseTimeMs != nil { + l = (*wrapperspb.Int64Value)(m.ResponseTimeMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Error != nil { + l = (*wrapperspb.StringValue)(m.Error).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Identify) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RemotePeer != nil { + l = (*wrapperspb.StringValue)(m.RemotePeer).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Success != nil { + l = (*wrapperspb.BoolValue)(m.Success).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Error != nil { + l = (*wrapperspb.StringValue)(m.Error).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AgentVersion != nil { + l = (*wrapperspb.StringValue)(m.AgentVersion).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProtocolVersion != nil { + l = (*wrapperspb.StringValue)(m.ProtocolVersion).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Protocols) > 0 { + for _, s := range m.Protocols { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.ListenAddrs) > 0 { + for _, s := range m.ListenAddrs { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.ObservedAddr != nil { + l = (*wrapperspb.StringValue)(m.ObservedAddr).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Transport != nil { + l = (*wrapperspb.StringValue)(m.Transport).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Security != nil { + l = (*wrapperspb.StringValue)(m.Security).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Muxer != nil { + l = (*wrapperspb.StringValue)(m.Muxer).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Direction != nil { + l = (*wrapperspb.StringValue)(m.Direction).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RemoteMultiaddr != nil { + l = (*wrapperspb.StringValue)(m.RemoteMultiaddr).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AddPeer) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AddPeer: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AddPeer: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Protocol", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Protocol == nil { + m.Protocol = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Protocol).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RemovePeer) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RemovePeer: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RemovePeer: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RecvRPC) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RecvRPC: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RecvRPC: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Meta", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Meta == nil { + m.Meta = &RPCMeta{} + } + if err := m.Meta.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SendRPC) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SendRPC: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SendRPC: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Meta", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Meta == nil { + m.Meta = &RPCMeta{} + } + if err := m.Meta.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DropRPC) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DropRPC: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DropRPC: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Meta", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Meta == nil { + m.Meta = &RPCMeta{} + } + if err := m.Meta.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Join) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Join: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Join: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Leave) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Leave: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Leave: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Graft) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Graft: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Graft: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Prune) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Prune: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Prune: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *PublishMessage) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: PublishMessage: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PublishMessage: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MsgId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MsgId == nil { + m.MsgId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MsgId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RejectMessage) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RejectMessage: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RejectMessage: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MsgId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MsgId == nil { + m.MsgId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MsgId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Reason", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Reason == nil { + m.Reason = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Reason).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Local", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Local == nil { + m.Local = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Local).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MsgSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MsgSize == nil { + m.MsgSize = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MsgSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SeqNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SeqNumber == nil { + m.SeqNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SeqNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DuplicateMessage) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DuplicateMessage: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DuplicateMessage: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MsgId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MsgId == nil { + m.MsgId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MsgId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Local", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Local == nil { + m.Local = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Local).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MsgSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MsgSize == nil { + m.MsgSize = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MsgSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SeqNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SeqNumber == nil { + m.SeqNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SeqNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DeliverMessage) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DeliverMessage: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeliverMessage: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MsgId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MsgId == nil { + m.MsgId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MsgId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Local", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Local == nil { + m.Local = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Local).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MsgSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MsgSize == nil { + m.MsgSize = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MsgSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SeqNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SeqNumber == nil { + m.SeqNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SeqNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *TraceEventMetadata) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TraceEventMetadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TraceEventMetadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Connected) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Connected: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Connected: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemotePeer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemotePeer == nil { + m.RemotePeer = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RemotePeer).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemoteMaddrs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemoteMaddrs == nil { + m.RemoteMaddrs = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RemoteMaddrs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentVersion", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AgentVersion == nil { + m.AgentVersion = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.AgentVersion).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Direction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Direction == nil { + m.Direction = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Direction).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Opened", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Opened == nil { + m.Opened = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.Opened).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transient", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Transient == nil { + m.Transient = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Transient).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Limited", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Limited == nil { + m.Limited = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Limited).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Disconnected) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Disconnected: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Disconnected: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemotePeer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemotePeer == nil { + m.RemotePeer = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RemotePeer).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemoteMaddrs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemoteMaddrs == nil { + m.RemoteMaddrs = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RemoteMaddrs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentVersion", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AgentVersion == nil { + m.AgentVersion = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.AgentVersion).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Direction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Direction == nil { + m.Direction = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Direction).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Opened", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Opened == nil { + m.Opened = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.Opened).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transient", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Transient == nil { + m.Transient = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Transient).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Limited", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Limited == nil { + m.Limited = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Limited).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *HandleStatus) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: HandleStatus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HandleStatus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProtocolId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProtocolId == nil { + m.ProtocolId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ProtocolId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Error == nil { + m.Error = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Error).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Latency", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Latency == nil { + m.Latency = &wrapperspb1.FloatValue{} + } + if err := (*wrapperspb.FloatValue)(m.Latency).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Request", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Request == nil { + m.Request = &Status{} + } + if err := m.Request.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Response", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Response == nil { + m.Response = &Status{} + } + if err := m.Response.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Direction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Direction == nil { + m.Direction = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Direction).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *HandleMetadata) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: HandleMetadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HandleMetadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProtocolId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProtocolId == nil { + m.ProtocolId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ProtocolId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Error == nil { + m.Error = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Error).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Latency", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Latency == nil { + m.Latency = &wrapperspb1.FloatValue{} + } + if err := (*wrapperspb.FloatValue)(m.Latency).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &Metadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Direction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Direction == nil { + m.Direction = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Direction).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyntheticHeartbeat) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyntheticHeartbeat: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyntheticHeartbeat: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemotePeer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemotePeer == nil { + m.RemotePeer = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RemotePeer).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemoteMaddrs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemoteMaddrs == nil { + m.RemoteMaddrs = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RemoteMaddrs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LatencyMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.LatencyMs == nil { + m.LatencyMs = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.LatencyMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentVersion", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AgentVersion == nil { + m.AgentVersion = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.AgentVersion).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Direction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Direction == nil { + m.Direction = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Direction).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Protocols", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Protocols = append(m.Protocols, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectionAgeNs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ConnectionAgeNs == nil { + m.ConnectionAgeNs = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.ConnectionAgeNs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RPCMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RPCMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RPCMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Messages", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Messages = append(m.Messages, &MessageMeta{}) + if err := m.Messages[len(m.Messages)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Subscriptions", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Subscriptions = append(m.Subscriptions, &SubMeta{}) + if err := m.Subscriptions[len(m.Subscriptions)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Control", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Control == nil { + m.Control = &ControlMeta{} + } + if err := m.Control.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MessageMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: MessageMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MessageMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TopicId == nil { + m.TopicId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TopicId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MessageMetaItem) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: MessageMetaItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MessageMetaItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RootEventId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RootEventId == nil { + m.RootEventId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RootEventId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TopicId == nil { + m.TopicId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TopicId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ControlIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ControlIndex == nil { + m.ControlIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ControlIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SubMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SubMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SubMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Subscribe", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Subscribe == nil { + m.Subscribe = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Subscribe).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TopicId == nil { + m.TopicId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TopicId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SubMetaItem) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SubMetaItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SubMetaItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RootEventId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RootEventId == nil { + m.RootEventId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RootEventId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TopicId == nil { + m.TopicId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TopicId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Subscribe", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Subscribe == nil { + m.Subscribe = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Subscribe).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ControlIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ControlIndex == nil { + m.ControlIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ControlIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Ihave", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Ihave = append(m.Ihave, &ControlIHaveMeta{}) + if err := m.Ihave[len(m.Ihave)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Iwant", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Iwant = append(m.Iwant, &ControlIWantMeta{}) + if err := m.Iwant[len(m.Iwant)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Graft", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Graft = append(m.Graft, &ControlGraftMeta{}) + if err := m.Graft[len(m.Graft)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Prune", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Prune = append(m.Prune, &ControlPruneMeta{}) + if err := m.Prune[len(m.Prune)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Idontwant", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Idontwant = append(m.Idontwant, &ControlIDontWantMeta{}) + if err := m.Idontwant[len(m.Idontwant)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlIHaveMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlIHaveMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlIHaveMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TopicId == nil { + m.TopicId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TopicId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageIds", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + if err := (*wrapperspb.StringValue)(m.MessageIds[len(m.MessageIds)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlIHaveMetaItem) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlIHaveMetaItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlIHaveMetaItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RootEventId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RootEventId == nil { + m.RootEventId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RootEventId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageIndex == nil { + m.MessageIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MessageIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ControlIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ControlIndex == nil { + m.ControlIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ControlIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlIWantMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlIWantMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlIWantMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageIds", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + if err := (*wrapperspb.StringValue)(m.MessageIds[len(m.MessageIds)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlIWantMetaItem) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlIWantMetaItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlIWantMetaItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RootEventId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RootEventId == nil { + m.RootEventId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RootEventId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageIndex == nil { + m.MessageIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MessageIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ControlIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ControlIndex == nil { + m.ControlIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ControlIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlGraftMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlGraftMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlGraftMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TopicId == nil { + m.TopicId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TopicId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlGraftMetaItem) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlGraftMetaItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlGraftMetaItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RootEventId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RootEventId == nil { + m.RootEventId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RootEventId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ControlIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ControlIndex == nil { + m.ControlIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ControlIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlPruneMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlPruneMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlPruneMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TopicId == nil { + m.TopicId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TopicId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerIds", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PeerIds = append(m.PeerIds, &wrapperspb1.StringValue{}) + if err := (*wrapperspb.StringValue)(m.PeerIds[len(m.PeerIds)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlPruneMetaItem) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlPruneMetaItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlPruneMetaItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RootEventId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RootEventId == nil { + m.RootEventId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RootEventId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GraftPeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GraftPeerId == nil { + m.GraftPeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.GraftPeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ControlIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ControlIndex == nil { + m.ControlIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ControlIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerIndex == nil { + m.PeerIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.PeerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlIDontWantMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlIDontWantMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlIDontWantMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageIds", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + if err := (*wrapperspb.StringValue)(m.MessageIds[len(m.MessageIds)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ControlIDontWantMetaItem) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ControlIDontWantMetaItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ControlIDontWantMetaItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RootEventId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RootEventId == nil { + m.RootEventId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RootEventId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageIndex == nil { + m.MessageIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MessageIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ControlIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ControlIndex == nil { + m.ControlIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ControlIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DataColumnCustodyProbe) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DataColumnCustodyProbe: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DataColumnCustodyProbe: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field JobStartTimestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.JobStartTimestamp == nil { + m.JobStartTimestamp = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.JobStartTimestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SlotStartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SlotStartDateTime == nil { + m.SlotStartDateTime = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SlotStartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Epoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EpochStartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EpochStartDateTime == nil { + m.EpochStartDateTime = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.EpochStartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockRequestSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockRequestSlot == nil { + m.WallclockRequestSlot = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.WallclockRequestSlot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockRequestSlotStartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockRequestSlotStartDateTime == nil { + m.WallclockRequestSlotStartDateTime = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.WallclockRequestSlotStartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockRequestEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockRequestEpoch == nil { + m.WallclockRequestEpoch = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.WallclockRequestEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockRequestEpochStartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockRequestEpochStartDateTime == nil { + m.WallclockRequestEpochStartDateTime = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.WallclockRequestEpochStartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ColumnIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ColumnIndex == nil { + m.ColumnIndex = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ColumnIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ColumnRowsCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ColumnRowsCount == nil { + m.ColumnRowsCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ColumnRowsCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeaconBlockRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BeaconBlockRoot == nil { + m.BeaconBlockRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.BeaconBlockRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Result == nil { + m.Result = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Result).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResponseTimeMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ResponseTimeMs == nil { + m.ResponseTimeMs = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.ResponseTimeMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Error == nil { + m.Error = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Error).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Identify) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Identify: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Identify: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemotePeer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemotePeer == nil { + m.RemotePeer = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RemotePeer).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Success", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Success == nil { + m.Success = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Success).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Error == nil { + m.Error = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Error).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentVersion", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AgentVersion == nil { + m.AgentVersion = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.AgentVersion).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProtocolVersion", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProtocolVersion == nil { + m.ProtocolVersion = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ProtocolVersion).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Protocols", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Protocols = append(m.Protocols, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ListenAddrs", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ListenAddrs = append(m.ListenAddrs, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ObservedAddr", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ObservedAddr == nil { + m.ObservedAddr = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ObservedAddr).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transport", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Transport == nil { + m.Transport = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Transport).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Security", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Security == nil { + m.Security = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Security).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Muxer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Muxer == nil { + m.Muxer = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Muxer).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Direction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Direction == nil { + m.Direction = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Direction).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemoteMultiaddr", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemoteMultiaddr == nil { + m.RemoteMultiaddr = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.RemoteMultiaddr).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/mevrelay/bids_vtproto.pb.go b/pkg/proto/mevrelay/bids_vtproto.pb.go new file mode 100644 index 00000000..d8bc60d2 --- /dev/null +++ b/pkg/proto/mevrelay/bids_vtproto.pb.go @@ -0,0 +1,816 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/mevrelay/bids.proto + +package mevrelay + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *BidTrace) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BidTrace) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BidTrace) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.OptimisticSubmission != nil { + size, err := (*wrapperspb.BoolValue)(m.OptimisticSubmission).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x72 + } + if m.TimestampMs != nil { + size, err := (*wrapperspb.Int64Value)(m.TimestampMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.Int64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.NumTx != nil { + size, err := (*wrapperspb.UInt64Value)(m.NumTx).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.Value != nil { + size, err := (*wrapperspb.StringValue)(m.Value).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.ProposerFeeRecipient != nil { + size, err := (*wrapperspb.StringValue)(m.ProposerFeeRecipient).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.ProposerPubkey != nil { + size, err := (*wrapperspb.StringValue)(m.ProposerPubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.BuilderPubkey != nil { + size, err := (*wrapperspb.StringValue)(m.BuilderPubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.BlockHash != nil { + size, err := (*wrapperspb.StringValue)(m.BlockHash).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.ParentHash != nil { + size, err := (*wrapperspb.StringValue)(m.ParentHash).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BidTrace) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ParentHash != nil { + l = (*wrapperspb.StringValue)(m.ParentHash).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockHash != nil { + l = (*wrapperspb.StringValue)(m.BlockHash).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BuilderPubkey != nil { + l = (*wrapperspb.StringValue)(m.BuilderPubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerPubkey != nil { + l = (*wrapperspb.StringValue)(m.ProposerPubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerFeeRecipient != nil { + l = (*wrapperspb.StringValue)(m.ProposerFeeRecipient).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Value != nil { + l = (*wrapperspb.StringValue)(m.Value).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.NumTx != nil { + l = (*wrapperspb.UInt64Value)(m.NumTx).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.Int64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TimestampMs != nil { + l = (*wrapperspb.Int64Value)(m.TimestampMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.OptimisticSubmission != nil { + l = (*wrapperspb.BoolValue)(m.OptimisticSubmission).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BidTrace) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BidTrace: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BidTrace: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ParentHash == nil { + m.ParentHash = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ParentHash).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockHash == nil { + m.BlockHash = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.BlockHash).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BuilderPubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BuilderPubkey == nil { + m.BuilderPubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.BuilderPubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerPubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerPubkey == nil { + m.ProposerPubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ProposerPubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerFeeRecipient", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerFeeRecipient == nil { + m.ProposerFeeRecipient = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ProposerFeeRecipient).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Value == nil { + m.Value = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Value).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NumTx", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NumTx == nil { + m.NumTx = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.NumTx).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TimestampMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TimestampMs == nil { + m.TimestampMs = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.TimestampMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OptimisticSubmission", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.OptimisticSubmission == nil { + m.OptimisticSubmission = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.OptimisticSubmission).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/mevrelay/payloads_vtproto.pb.go b/pkg/proto/mevrelay/payloads_vtproto.pb.go new file mode 100644 index 00000000..9db1d5d6 --- /dev/null +++ b/pkg/proto/mevrelay/payloads_vtproto.pb.go @@ -0,0 +1,666 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/mevrelay/payloads.proto + +package mevrelay + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *ProposerPayloadDelivered) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ProposerPayloadDelivered) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ProposerPayloadDelivered) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.NumTx != nil { + size, err := (*wrapperspb.UInt64Value)(m.NumTx).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.Value != nil { + size, err := (*wrapperspb.StringValue)(m.Value).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.ProposerFeeRecipient != nil { + size, err := (*wrapperspb.StringValue)(m.ProposerFeeRecipient).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.ProposerPubkey != nil { + size, err := (*wrapperspb.StringValue)(m.ProposerPubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.BuilderPubkey != nil { + size, err := (*wrapperspb.StringValue)(m.BuilderPubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.BlockHash != nil { + size, err := (*wrapperspb.StringValue)(m.BlockHash).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.ParentHash != nil { + size, err := (*wrapperspb.StringValue)(m.ParentHash).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ProposerPayloadDelivered) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ParentHash != nil { + l = (*wrapperspb.StringValue)(m.ParentHash).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockHash != nil { + l = (*wrapperspb.StringValue)(m.BlockHash).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BuilderPubkey != nil { + l = (*wrapperspb.StringValue)(m.BuilderPubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerPubkey != nil { + l = (*wrapperspb.StringValue)(m.ProposerPubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerFeeRecipient != nil { + l = (*wrapperspb.StringValue)(m.ProposerFeeRecipient).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Value != nil { + l = (*wrapperspb.StringValue)(m.Value).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.NumTx != nil { + l = (*wrapperspb.UInt64Value)(m.NumTx).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ProposerPayloadDelivered) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ProposerPayloadDelivered: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ProposerPayloadDelivered: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ParentHash == nil { + m.ParentHash = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ParentHash).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockHash == nil { + m.BlockHash = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.BlockHash).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BuilderPubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BuilderPubkey == nil { + m.BuilderPubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.BuilderPubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerPubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerPubkey == nil { + m.ProposerPubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ProposerPubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerFeeRecipient", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerFeeRecipient == nil { + m.ProposerFeeRecipient = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ProposerFeeRecipient).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Value == nil { + m.Value = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Value).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NumTx", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NumTx == nil { + m.NumTx = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.NumTx).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/mevrelay/relay_vtproto.pb.go b/pkg/proto/mevrelay/relay_vtproto.pb.go new file mode 100644 index 00000000..c2e7b4f3 --- /dev/null +++ b/pkg/proto/mevrelay/relay_vtproto.pb.go @@ -0,0 +1,704 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/mevrelay/relay.proto + +package mevrelay + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *Relay) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Relay) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Relay) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Url != nil { + size, err := (*wrapperspb.StringValue)(m.Url).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Name != nil { + size, err := (*wrapperspb.StringValue)(m.Name).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ValidatorRegistrationMessage) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ValidatorRegistrationMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ValidatorRegistrationMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Pubkey != nil { + size, err := (*wrapperspb.StringValue)(m.Pubkey).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Timestamp != nil { + size, err := (*wrapperspb.UInt64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.FeeRecipient != nil { + size, err := (*wrapperspb.StringValue)(m.FeeRecipient).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ValidatorRegistration) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ValidatorRegistration) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ValidatorRegistration) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Signature != nil { + size, err := (*wrapperspb.StringValue)(m.Signature).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Message != nil { + size, err := m.Message.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Relay) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Name != nil { + l = (*wrapperspb.StringValue)(m.Name).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Url != nil { + l = (*wrapperspb.StringValue)(m.Url).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ValidatorRegistrationMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.FeeRecipient != nil { + l = (*wrapperspb.StringValue)(m.FeeRecipient).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.UInt64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Pubkey != nil { + l = (*wrapperspb.StringValue)(m.Pubkey).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ValidatorRegistration) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Message != nil { + l = m.Message.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Signature != nil { + l = (*wrapperspb.StringValue)(m.Signature).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Relay) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Relay: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Relay: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Name == nil { + m.Name = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Name).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Url", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Url == nil { + m.Url = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Url).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ValidatorRegistrationMessage) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ValidatorRegistrationMessage: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ValidatorRegistrationMessage: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FeeRecipient", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FeeRecipient == nil { + m.FeeRecipient = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.FeeRecipient).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pubkey", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Pubkey == nil { + m.Pubkey = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Pubkey).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ValidatorRegistration) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ValidatorRegistration: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ValidatorRegistration: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Message == nil { + m.Message = &ValidatorRegistrationMessage{} + } + if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Signature == nil { + m.Signature = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Signature).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/noderecord/consensus_vtproto.pb.go b/pkg/proto/noderecord/consensus_vtproto.pb.go new file mode 100644 index 00000000..35ce3b6a --- /dev/null +++ b/pkg/proto/noderecord/consensus_vtproto.pb.go @@ -0,0 +1,1076 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/noderecord/consensus.proto + +package noderecord + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + timestamppb "github.com/planetscale/vtprotobuf/types/known/timestamppb" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *Consensus) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Consensus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Consensus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.HasIpv6 != nil { + size, err := (*wrapperspb.BoolValue)(m.HasIpv6).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x9a + } + if m.Quic != nil { + size, err := (*wrapperspb.UInt32Value)(m.Quic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x92 + } + if m.Udp != nil { + size, err := (*wrapperspb.UInt32Value)(m.Udp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + if m.Tcp != nil { + size, err := (*wrapperspb.UInt32Value)(m.Tcp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if m.Ip != nil { + size, err := (*wrapperspb.StringValue)(m.Ip).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + if m.Cgc != nil { + size, err := (*wrapperspb.StringValue)(m.Cgc).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x72 + } + if m.HeadSlotStartDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.HeadSlotStartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if m.HeadSlot != nil { + size, err := (*wrapperspb.UInt64Value)(m.HeadSlot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.HeadRoot != nil { + size, err := (*wrapperspb.StringValue)(m.HeadRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.FinalizedEpochStartDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.FinalizedEpochStartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.FinalizedEpoch != nil { + size, err := (*wrapperspb.UInt64Value)(m.FinalizedEpoch).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.FinalizedRoot != nil { + size, err := (*wrapperspb.StringValue)(m.FinalizedRoot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.NextForkDigest != nil { + size, err := (*wrapperspb.StringValue)(m.NextForkDigest).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.ForkDigest != nil { + size, err := (*wrapperspb.StringValue)(m.ForkDigest).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Name != nil { + size, err := (*wrapperspb.StringValue)(m.Name).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Timestamp != nil { + size, err := (*wrapperspb.Int64Value)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.PeerId != nil { + size, err := (*wrapperspb.StringValue)(m.PeerId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.NodeId != nil { + size, err := (*wrapperspb.StringValue)(m.NodeId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Enr != nil { + size, err := (*wrapperspb.StringValue)(m.Enr).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Consensus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Enr != nil { + l = (*wrapperspb.StringValue)(m.Enr).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.NodeId != nil { + l = (*wrapperspb.StringValue)(m.NodeId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PeerId != nil { + l = (*wrapperspb.StringValue)(m.PeerId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*wrapperspb.Int64Value)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Name != nil { + l = (*wrapperspb.StringValue)(m.Name).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ForkDigest != nil { + l = (*wrapperspb.StringValue)(m.ForkDigest).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.NextForkDigest != nil { + l = (*wrapperspb.StringValue)(m.NextForkDigest).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedRoot != nil { + l = (*wrapperspb.StringValue)(m.FinalizedRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedEpoch != nil { + l = (*wrapperspb.UInt64Value)(m.FinalizedEpoch).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedEpochStartDateTime != nil { + l = (*timestamppb.Timestamp)(m.FinalizedEpochStartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HeadRoot != nil { + l = (*wrapperspb.StringValue)(m.HeadRoot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HeadSlot != nil { + l = (*wrapperspb.UInt64Value)(m.HeadSlot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HeadSlotStartDateTime != nil { + l = (*timestamppb.Timestamp)(m.HeadSlotStartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Cgc != nil { + l = (*wrapperspb.StringValue)(m.Cgc).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Ip != nil { + l = (*wrapperspb.StringValue)(m.Ip).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Tcp != nil { + l = (*wrapperspb.UInt32Value)(m.Tcp).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Udp != nil { + l = (*wrapperspb.UInt32Value)(m.Udp).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Quic != nil { + l = (*wrapperspb.UInt32Value)(m.Quic).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HasIpv6 != nil { + l = (*wrapperspb.BoolValue)(m.HasIpv6).SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Consensus) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Consensus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Consensus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Enr", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Enr == nil { + m.Enr = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Enr).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NodeId == nil { + m.NodeId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.NodeId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PeerId == nil { + m.PeerId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.PeerId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Name == nil { + m.Name = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Name).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkDigest", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ForkDigest == nil { + m.ForkDigest = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ForkDigest).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NextForkDigest", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NextForkDigest == nil { + m.NextForkDigest = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.NextForkDigest).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedRoot == nil { + m.FinalizedRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.FinalizedRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedEpoch == nil { + m.FinalizedEpoch = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.FinalizedEpoch).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpochStartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedEpochStartDateTime == nil { + m.FinalizedEpochStartDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.FinalizedEpochStartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadRoot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HeadRoot == nil { + m.HeadRoot = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.HeadRoot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HeadSlot == nil { + m.HeadSlot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.HeadSlot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadSlotStartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HeadSlotStartDateTime == nil { + m.HeadSlotStartDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.HeadSlotStartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cgc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Cgc == nil { + m.Cgc = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Cgc).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Ip", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Ip == nil { + m.Ip = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Ip).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tcp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Tcp == nil { + m.Tcp = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Tcp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Udp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Udp == nil { + m.Udp = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Udp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 18: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Quic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Quic == nil { + m.Quic = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Quic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 19: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HasIpv6", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HasIpv6 == nil { + m.HasIpv6 = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.HasIpv6).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/noderecord/execution_vtproto.pb.go b/pkg/proto/noderecord/execution_vtproto.pb.go new file mode 100644 index 00000000..598681c6 --- /dev/null +++ b/pkg/proto/noderecord/execution_vtproto.pb.go @@ -0,0 +1,868 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/noderecord/execution.proto + +package noderecord + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + timestamppb "github.com/planetscale/vtprotobuf/types/known/timestamppb" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" +) + +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) +) + +func (m *Execution) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Execution) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Execution) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.HasIpv6 != nil { + size, err := (*wrapperspb.BoolValue)(m.HasIpv6).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + if m.Udp != nil { + size, err := (*wrapperspb.UInt32Value)(m.Udp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x72 + } + if m.Tcp != nil { + size, err := (*wrapperspb.UInt32Value)(m.Tcp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if m.Ip != nil { + size, err := (*wrapperspb.StringValue)(m.Ip).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.NodeId != nil { + size, err := (*wrapperspb.StringValue)(m.NodeId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.ForkIdNext != nil { + size, err := (*wrapperspb.StringValue)(m.ForkIdNext).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.ForkIdHash != nil { + size, err := (*wrapperspb.StringValue)(m.ForkIdHash).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.Genesis != nil { + size, err := (*wrapperspb.StringValue)(m.Genesis).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.Head != nil { + size, err := (*wrapperspb.StringValue)(m.Head).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.TotalDifficulty != nil { + size, err := (*wrapperspb.StringValue)(m.TotalDifficulty).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.ProtocolVersion != nil { + size, err := (*wrapperspb.StringValue)(m.ProtocolVersion).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Capabilities != nil { + size, err := (*wrapperspb.StringValue)(m.Capabilities).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Name != nil { + size, err := (*wrapperspb.StringValue)(m.Name).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Timestamp != nil { + size, err := (*timestamppb.Timestamp)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Enr != nil { + size, err := (*wrapperspb.StringValue)(m.Enr).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Execution) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Enr != nil { + l = (*wrapperspb.StringValue)(m.Enr).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*timestamppb.Timestamp)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Name != nil { + l = (*wrapperspb.StringValue)(m.Name).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Capabilities != nil { + l = (*wrapperspb.StringValue)(m.Capabilities).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProtocolVersion != nil { + l = (*wrapperspb.StringValue)(m.ProtocolVersion).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TotalDifficulty != nil { + l = (*wrapperspb.StringValue)(m.TotalDifficulty).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Head != nil { + l = (*wrapperspb.StringValue)(m.Head).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Genesis != nil { + l = (*wrapperspb.StringValue)(m.Genesis).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ForkIdHash != nil { + l = (*wrapperspb.StringValue)(m.ForkIdHash).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ForkIdNext != nil { + l = (*wrapperspb.StringValue)(m.ForkIdNext).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.NodeId != nil { + l = (*wrapperspb.StringValue)(m.NodeId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Ip != nil { + l = (*wrapperspb.StringValue)(m.Ip).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Tcp != nil { + l = (*wrapperspb.UInt32Value)(m.Tcp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Udp != nil { + l = (*wrapperspb.UInt32Value)(m.Udp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HasIpv6 != nil { + l = (*wrapperspb.BoolValue)(m.HasIpv6).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Execution) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Execution: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Execution: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Enr", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Enr == nil { + m.Enr = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Enr).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Name == nil { + m.Name = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Name).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Capabilities", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Capabilities == nil { + m.Capabilities = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Capabilities).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProtocolVersion", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProtocolVersion == nil { + m.ProtocolVersion = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ProtocolVersion).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalDifficulty", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TotalDifficulty == nil { + m.TotalDifficulty = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.TotalDifficulty).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Head", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Head == nil { + m.Head = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Head).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Genesis", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Genesis == nil { + m.Genesis = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Genesis).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkIdHash", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ForkIdHash == nil { + m.ForkIdHash = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ForkIdHash).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkIdNext", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ForkIdNext == nil { + m.ForkIdNext = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.ForkIdNext).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NodeId == nil { + m.NodeId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.NodeId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Ip", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Ip == nil { + m.Ip = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Ip).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tcp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Tcp == nil { + m.Tcp = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Tcp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Udp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Udp == nil { + m.Udp = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Udp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HasIpv6", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HasIpv6 == nil { + m.HasIpv6 = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.HasIpv6).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/xatu/coordinator_vtproto.pb.go b/pkg/proto/xatu/coordinator_vtproto.pb.go new file mode 100644 index 00000000..003462ba --- /dev/null +++ b/pkg/proto/xatu/coordinator_vtproto.pb.go @@ -0,0 +1,11069 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/xatu/coordinator.proto + +package xatu + +import ( + fmt "fmt" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + timestamppb "github.com/planetscale/vtprotobuf/types/known/timestamppb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" + io "io" +) + +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) +) + +func (m *CreateNodeRecordsRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateNodeRecordsRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateNodeRecordsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.NodeRecords) > 0 { + for iNdEx := len(m.NodeRecords) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.NodeRecords[iNdEx]) + copy(dAtA[i:], m.NodeRecords[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecords[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *CreateNodeRecordsResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateNodeRecordsResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateNodeRecordsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + return len(dAtA) - i, nil +} + +func (m *ListStalledExecutionNodeRecordsRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ListStalledExecutionNodeRecordsRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ListStalledExecutionNodeRecordsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.PageSize != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.PageSize)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ListStalledExecutionNodeRecordsResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ListStalledExecutionNodeRecordsResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ListStalledExecutionNodeRecordsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.NodeRecords) > 0 { + for iNdEx := len(m.NodeRecords) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.NodeRecords[iNdEx]) + copy(dAtA[i:], m.NodeRecords[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecords[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *ExecutionNodeStatus_Capability) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionNodeStatus_Capability) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionNodeStatus_Capability) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Version != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Version)) + i-- + dAtA[i] = 0x10 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionNodeStatus_ForkID) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionNodeStatus_ForkID) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionNodeStatus_ForkID) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Next != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Next)) + i-- + dAtA[i] = 0x10 + } + if len(m.Hash) > 0 { + i -= len(m.Hash) + copy(dAtA[i:], m.Hash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Hash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionNodeStatus) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionNodeStatus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionNodeStatus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ForkId != nil { + size, err := m.ForkId.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if len(m.Genesis) > 0 { + i -= len(m.Genesis) + copy(dAtA[i:], m.Genesis) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Genesis))) + i-- + dAtA[i] = 0x42 + } + if len(m.Head) > 0 { + i -= len(m.Head) + copy(dAtA[i:], m.Head) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Head))) + i-- + dAtA[i] = 0x3a + } + if len(m.TotalDifficulty) > 0 { + i -= len(m.TotalDifficulty) + copy(dAtA[i:], m.TotalDifficulty) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.TotalDifficulty))) + i-- + dAtA[i] = 0x32 + } + if m.NetworkId != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.NetworkId)) + i-- + dAtA[i] = 0x28 + } + if m.ProtocolVersion != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ProtocolVersion)) + i-- + dAtA[i] = 0x20 + } + if len(m.Capabilities) > 0 { + for iNdEx := len(m.Capabilities) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Capabilities[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0x12 + } + if len(m.NodeRecord) > 0 { + i -= len(m.NodeRecord) + copy(dAtA[i:], m.NodeRecord) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecord))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CreateExecutionNodeRecordStatusRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateExecutionNodeRecordStatusRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateExecutionNodeRecordStatusRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Status != nil { + size, err := m.Status.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CreateExecutionNodeRecordStatusResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateExecutionNodeRecordStatusResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateExecutionNodeRecordStatusResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + return len(dAtA) - i, nil +} + +func (m *CoordinatedNodeRecord) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CoordinatedNodeRecord) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CoordinatedNodeRecord) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ConnectionAttempts != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ConnectionAttempts)) + i-- + dAtA[i] = 0x18 + } + if m.Connected { + i-- + if m.Connected { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if len(m.NodeRecord) > 0 { + i -= len(m.NodeRecord) + copy(dAtA[i:], m.NodeRecord) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecord))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CoordinateExecutionNodeRecordsRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CoordinateExecutionNodeRecordsRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CoordinateExecutionNodeRecordsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Capabilities) > 0 { + for iNdEx := len(m.Capabilities) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Capabilities[iNdEx]) + copy(dAtA[i:], m.Capabilities[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Capabilities[iNdEx]))) + i-- + dAtA[i] = 0x32 + } + } + if len(m.ClientId) > 0 { + i -= len(m.ClientId) + copy(dAtA[i:], m.ClientId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ClientId))) + i-- + dAtA[i] = 0x2a + } + if len(m.ForkIdHashes) > 0 { + for iNdEx := len(m.ForkIdHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ForkIdHashes[iNdEx]) + copy(dAtA[i:], m.ForkIdHashes[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ForkIdHashes[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + if len(m.NetworkIds) > 0 { + var pksize2 int + for _, num := range m.NetworkIds { + pksize2 += protohelpers.SizeOfVarint(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num := range m.NetworkIds { + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = protohelpers.EncodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0x1a + } + if m.Limit != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Limit)) + i-- + dAtA[i] = 0x10 + } + if len(m.NodeRecords) > 0 { + for iNdEx := len(m.NodeRecords) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.NodeRecords[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *CoordinateExecutionNodeRecordsResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CoordinateExecutionNodeRecordsResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CoordinateExecutionNodeRecordsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.RetryDelay != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.RetryDelay)) + i-- + dAtA[i] = 0x10 + } + if len(m.NodeRecords) > 0 { + for iNdEx := len(m.NodeRecords) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.NodeRecords[iNdEx]) + copy(dAtA[i:], m.NodeRecords[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecords[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *ConsensusNodeStatus) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ConsensusNodeStatus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ConsensusNodeStatus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.PeerId) > 0 { + i -= len(m.PeerId) + copy(dAtA[i:], m.PeerId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PeerId))) + i-- + dAtA[i] = 0x72 + } + if len(m.NodeId) > 0 { + i -= len(m.NodeId) + copy(dAtA[i:], m.NodeId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeId))) + i-- + dAtA[i] = 0x6a + } + if m.NetworkId != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.NetworkId)) + i-- + dAtA[i] = 0x60 + } + if len(m.Cgc) > 0 { + i -= len(m.Cgc) + copy(dAtA[i:], m.Cgc) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Cgc))) + i-- + dAtA[i] = 0x5a + } + if m.HeadSlotStartDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.HeadSlotStartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if len(m.HeadSlot) > 0 { + i -= len(m.HeadSlot) + copy(dAtA[i:], m.HeadSlot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.HeadSlot))) + i-- + dAtA[i] = 0x4a + } + if len(m.HeadRoot) > 0 { + i -= len(m.HeadRoot) + copy(dAtA[i:], m.HeadRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.HeadRoot))) + i-- + dAtA[i] = 0x42 + } + if m.FinalizedEpochStartDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.FinalizedEpochStartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.FinalizedEpoch) > 0 { + i -= len(m.FinalizedEpoch) + copy(dAtA[i:], m.FinalizedEpoch) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FinalizedEpoch))) + i-- + dAtA[i] = 0x32 + } + if len(m.FinalizedRoot) > 0 { + i -= len(m.FinalizedRoot) + copy(dAtA[i:], m.FinalizedRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.FinalizedRoot))) + i-- + dAtA[i] = 0x2a + } + if len(m.NextForkDigest) > 0 { + i -= len(m.NextForkDigest) + copy(dAtA[i:], m.NextForkDigest) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NextForkDigest))) + i-- + dAtA[i] = 0x22 + } + if len(m.ForkDigest) > 0 { + i -= len(m.ForkDigest) + copy(dAtA[i:], m.ForkDigest) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ForkDigest))) + i-- + dAtA[i] = 0x1a + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0x12 + } + if len(m.NodeRecord) > 0 { + i -= len(m.NodeRecord) + copy(dAtA[i:], m.NodeRecord) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecord))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ListStalledConsensusNodeRecordsRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ListStalledConsensusNodeRecordsRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ListStalledConsensusNodeRecordsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.PageSize != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.PageSize)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ListStalledConsensusNodeRecordsResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ListStalledConsensusNodeRecordsResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ListStalledConsensusNodeRecordsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.NodeRecords) > 0 { + for iNdEx := len(m.NodeRecords) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.NodeRecords[iNdEx]) + copy(dAtA[i:], m.NodeRecords[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecords[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *CreateConsensusNodeRecordStatusRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateConsensusNodeRecordStatusRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateConsensusNodeRecordStatusRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Status != nil { + size, err := m.Status.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CreateConsensusNodeRecordStatusResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateConsensusNodeRecordStatusResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateConsensusNodeRecordStatusResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + return len(dAtA) - i, nil +} + +func (m *CreateConsensusNodeRecordStatusesRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateConsensusNodeRecordStatusesRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateConsensusNodeRecordStatusesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Statuses) > 0 { + for iNdEx := len(m.Statuses) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Statuses[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *CreateConsensusNodeRecordStatusesResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateConsensusNodeRecordStatusesResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateConsensusNodeRecordStatusesResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + return len(dAtA) - i, nil +} + +func (m *CoordinateConsensusNodeRecordsRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CoordinateConsensusNodeRecordsRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CoordinateConsensusNodeRecordsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Capabilities) > 0 { + for iNdEx := len(m.Capabilities) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Capabilities[iNdEx]) + copy(dAtA[i:], m.Capabilities[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Capabilities[iNdEx]))) + i-- + dAtA[i] = 0x32 + } + } + if len(m.ClientId) > 0 { + i -= len(m.ClientId) + copy(dAtA[i:], m.ClientId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ClientId))) + i-- + dAtA[i] = 0x2a + } + if len(m.ForkIdHashes) > 0 { + for iNdEx := len(m.ForkIdHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ForkIdHashes[iNdEx]) + copy(dAtA[i:], m.ForkIdHashes[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ForkIdHashes[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + if len(m.NetworkIds) > 0 { + var pksize2 int + for _, num := range m.NetworkIds { + pksize2 += protohelpers.SizeOfVarint(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num := range m.NetworkIds { + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = protohelpers.EncodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0x1a + } + if m.Limit != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Limit)) + i-- + dAtA[i] = 0x10 + } + if len(m.NodeRecords) > 0 { + for iNdEx := len(m.NodeRecords) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.NodeRecords[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *CoordinateConsensusNodeRecordsResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CoordinateConsensusNodeRecordsResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CoordinateConsensusNodeRecordsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.RetryDelay != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.RetryDelay)) + i-- + dAtA[i] = 0x10 + } + if len(m.NodeRecords) > 0 { + for iNdEx := len(m.NodeRecords) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.NodeRecords[iNdEx]) + copy(dAtA[i:], m.NodeRecords[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecords[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *GetDiscoveryNodeRecordRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetDiscoveryNodeRecordRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetDiscoveryNodeRecordRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ForkIdHashes) > 0 { + for iNdEx := len(m.ForkIdHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ForkIdHashes[iNdEx]) + copy(dAtA[i:], m.ForkIdHashes[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ForkIdHashes[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.NetworkIds) > 0 { + var pksize2 int + for _, num := range m.NetworkIds { + pksize2 += protohelpers.SizeOfVarint(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num := range m.NetworkIds { + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = protohelpers.EncodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *GetDiscoveryNodeRecordResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetDiscoveryNodeRecordResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetDiscoveryNodeRecordResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.NodeRecord) > 0 { + i -= len(m.NodeRecord) + copy(dAtA[i:], m.NodeRecord) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecord))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *GetDiscoveryExecutionNodeRecordRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetDiscoveryExecutionNodeRecordRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetDiscoveryExecutionNodeRecordRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ForkIdHashes) > 0 { + for iNdEx := len(m.ForkIdHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ForkIdHashes[iNdEx]) + copy(dAtA[i:], m.ForkIdHashes[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ForkIdHashes[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.NetworkIds) > 0 { + var pksize2 int + for _, num := range m.NetworkIds { + pksize2 += protohelpers.SizeOfVarint(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num := range m.NetworkIds { + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = protohelpers.EncodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *GetDiscoveryExecutionNodeRecordResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetDiscoveryExecutionNodeRecordResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetDiscoveryExecutionNodeRecordResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.NodeRecord) > 0 { + i -= len(m.NodeRecord) + copy(dAtA[i:], m.NodeRecord) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecord))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *GetDiscoveryConsensusNodeRecordRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetDiscoveryConsensusNodeRecordRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetDiscoveryConsensusNodeRecordRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ForkDigests) > 0 { + for iNdEx := len(m.ForkDigests) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ForkDigests[iNdEx]) + copy(dAtA[i:], m.ForkDigests[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ForkDigests[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.NetworkIds) > 0 { + var pksize2 int + for _, num := range m.NetworkIds { + pksize2 += protohelpers.SizeOfVarint(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num := range m.NetworkIds { + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = protohelpers.EncodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *GetDiscoveryConsensusNodeRecordResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetDiscoveryConsensusNodeRecordResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetDiscoveryConsensusNodeRecordResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.NodeRecord) > 0 { + i -= len(m.NodeRecord) + copy(dAtA[i:], m.NodeRecord) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NodeRecord))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BackfillingCheckpointMarker) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BackfillingCheckpointMarker) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BackfillingCheckpointMarker) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillEpoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.BackfillEpoch)) + i-- + dAtA[i] = 0x10 + } + if m.FinalizedEpoch != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.FinalizedEpoch)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockVoluntaryExit) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockVoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockVoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockProposerSlashing) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockProposerSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockProposerSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockDeposit) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockDeposit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockDeposit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockAttesterSlashing) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockAttesterSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockAttesterSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockBlsToExecutionChange) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockBlsToExecutionChange) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockBlsToExecutionChange) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockExecutionTransaction) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockExecutionTransaction) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockExecutionTransaction) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockWithdrawal) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockWithdrawal) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockWithdrawal) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlock) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV1BeaconBlobSidecar) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV1BeaconBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV1BeaconBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV1BeaconProposerDuty) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV1BeaconProposerDuty) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV1BeaconProposerDuty) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockElaboratedAttestation) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockElaboratedAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockElaboratedAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV1BeaconValidators) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV1BeaconValidators) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV1BeaconValidators) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV1BeaconCommittee) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV1BeaconCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV1BeaconCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV1BeaconSyncCommittee) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV1BeaconSyncCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV1BeaconSyncCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CannonLocationEthV2BeaconBlockSyncAggregate) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocationEthV2BeaconBlockSyncAggregate) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocationEthV2BeaconBlockSyncAggregate) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.BackfillingCheckpointMarker != nil { + size, err := m.BackfillingCheckpointMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CannonLocation) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CannonLocation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if vtmsg, ok := m.Data.(interface { + MarshalToSizedBufferVT([]byte) (int, error) + }); ok { + size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + } + if m.Type != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x10 + } + if len(m.NetworkId) > 0 { + i -= len(m.NetworkId) + copy(dAtA[i:], m.NetworkId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NetworkId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CannonLocation_EthV2BeaconBlockVoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockVoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockVoluntaryExit != nil { + size, err := m.EthV2BeaconBlockVoluntaryExit.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlockProposerSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockProposerSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockProposerSlashing != nil { + size, err := m.EthV2BeaconBlockProposerSlashing.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlockDeposit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockDeposit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockDeposit != nil { + size, err := m.EthV2BeaconBlockDeposit.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlockAttesterSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockAttesterSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockAttesterSlashing != nil { + size, err := m.EthV2BeaconBlockAttesterSlashing.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlockBlsToExecutionChange) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockBlsToExecutionChange) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockBlsToExecutionChange != nil { + size, err := m.EthV2BeaconBlockBlsToExecutionChange.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlockExecutionTransaction) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockExecutionTransaction) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockExecutionTransaction != nil { + size, err := m.EthV2BeaconBlockExecutionTransaction.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlockWithdrawal) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockWithdrawal) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockWithdrawal != nil { + size, err := m.EthV2BeaconBlockWithdrawal.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlock != nil { + size, err := m.EthV2BeaconBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV1BeaconBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV1BeaconBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconBlobSidecar != nil { + size, err := m.EthV1BeaconBlobSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV1BeaconProposerDuty) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV1BeaconProposerDuty) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconProposerDuty != nil { + size, err := m.EthV1BeaconProposerDuty.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlockElaboratedAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockElaboratedAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockElaboratedAttestation != nil { + size, err := m.EthV2BeaconBlockElaboratedAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x72 + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV1BeaconValidators) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV1BeaconValidators) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconValidators != nil { + size, err := m.EthV1BeaconValidators.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV1BeaconCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV1BeaconCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconCommittee != nil { + size, err := m.EthV1BeaconCommittee.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV1BeaconSyncCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV1BeaconSyncCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconSyncCommittee != nil { + size, err := m.EthV1BeaconSyncCommittee.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *CannonLocation_EthV2BeaconBlockSyncAggregate) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CannonLocation_EthV2BeaconBlockSyncAggregate) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockSyncAggregate != nil { + size, err := m.EthV2BeaconBlockSyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *GetCannonLocationRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetCannonLocationRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetCannonLocationRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Type != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x10 + } + if len(m.NetworkId) > 0 { + i -= len(m.NetworkId) + copy(dAtA[i:], m.NetworkId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.NetworkId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *GetCannonLocationResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetCannonLocationResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetCannonLocationResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Location != nil { + size, err := m.Location.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *UpsertCannonLocationRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *UpsertCannonLocationRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *UpsertCannonLocationRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Location != nil { + size, err := m.Location.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *UpsertCannonLocationResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *UpsertCannonLocationResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *UpsertCannonLocationResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + return len(dAtA) - i, nil +} + +func (m *RelayMonitorSlotMarker) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RelayMonitorSlotMarker) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RelayMonitorSlotMarker) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.CurrentSlot != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.CurrentSlot)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *RelayMonitorLocationBidTrace) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RelayMonitorLocationBidTrace) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RelayMonitorLocationBidTrace) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SlotMarker != nil { + size, err := m.SlotMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RelayMonitorLocationPayloadDelivered) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RelayMonitorLocationPayloadDelivered) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RelayMonitorLocationPayloadDelivered) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SlotMarker != nil { + size, err := m.SlotMarker.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RelayMonitorLocation) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RelayMonitorLocation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RelayMonitorLocation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if vtmsg, ok := m.Data.(interface { + MarshalToSizedBufferVT([]byte) (int, error) + }); ok { + size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + } + if m.Type != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x20 + } + if len(m.RelayName) > 0 { + i -= len(m.RelayName) + copy(dAtA[i:], m.RelayName) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RelayName))) + i-- + dAtA[i] = 0x1a + } + if len(m.MetaClientName) > 0 { + i -= len(m.MetaClientName) + copy(dAtA[i:], m.MetaClientName) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MetaClientName))) + i-- + dAtA[i] = 0x12 + } + if len(m.MetaNetworkName) > 0 { + i -= len(m.MetaNetworkName) + copy(dAtA[i:], m.MetaNetworkName) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MetaNetworkName))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RelayMonitorLocation_BidTrace) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RelayMonitorLocation_BidTrace) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.BidTrace != nil { + size, err := m.BidTrace.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + return len(dAtA) - i, nil +} +func (m *RelayMonitorLocation_PayloadDelivered) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RelayMonitorLocation_PayloadDelivered) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.PayloadDelivered != nil { + size, err := m.PayloadDelivered.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + return len(dAtA) - i, nil +} +func (m *GetRelayMonitorLocationRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetRelayMonitorLocationRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetRelayMonitorLocationRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Type != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x20 + } + if len(m.RelayName) > 0 { + i -= len(m.RelayName) + copy(dAtA[i:], m.RelayName) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.RelayName))) + i-- + dAtA[i] = 0x1a + } + if len(m.MetaClientName) > 0 { + i -= len(m.MetaClientName) + copy(dAtA[i:], m.MetaClientName) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MetaClientName))) + i-- + dAtA[i] = 0x12 + } + if len(m.MetaNetworkName) > 0 { + i -= len(m.MetaNetworkName) + copy(dAtA[i:], m.MetaNetworkName) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MetaNetworkName))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *GetRelayMonitorLocationResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetRelayMonitorLocationResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetRelayMonitorLocationResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Location != nil { + size, err := m.Location.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *UpsertRelayMonitorLocationRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *UpsertRelayMonitorLocationRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *UpsertRelayMonitorLocationRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Location != nil { + size, err := m.Location.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *UpsertRelayMonitorLocationResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *UpsertRelayMonitorLocationResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *UpsertRelayMonitorLocationResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + return len(dAtA) - i, nil +} + +func (m *CreateNodeRecordsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NodeRecords) > 0 { + for _, s := range m.NodeRecords { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *CreateNodeRecordsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *ListStalledExecutionNodeRecordsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PageSize != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.PageSize)) + } + n += len(m.unknownFields) + return n +} + +func (m *ListStalledExecutionNodeRecordsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NodeRecords) > 0 { + for _, s := range m.NodeRecords { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionNodeStatus_Capability) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Version != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Version)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionNodeStatus_ForkID) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Hash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Next != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Next)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionNodeStatus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NodeRecord) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Name) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Capabilities) > 0 { + for _, e := range m.Capabilities { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.ProtocolVersion != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ProtocolVersion)) + } + if m.NetworkId != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.NetworkId)) + } + l = len(m.TotalDifficulty) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Head) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Genesis) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ForkId != nil { + l = m.ForkId.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CreateExecutionNodeRecordStatusRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Status != nil { + l = m.Status.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CreateExecutionNodeRecordStatusResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *CoordinatedNodeRecord) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NodeRecord) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Connected { + n += 2 + } + if m.ConnectionAttempts != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ConnectionAttempts)) + } + n += len(m.unknownFields) + return n +} + +func (m *CoordinateExecutionNodeRecordsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NodeRecords) > 0 { + for _, e := range m.NodeRecords { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.Limit != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Limit)) + } + if len(m.NetworkIds) > 0 { + l = 0 + for _, e := range m.NetworkIds { + l += protohelpers.SizeOfVarint(uint64(e)) + } + n += 1 + protohelpers.SizeOfVarint(uint64(l)) + l + } + if len(m.ForkIdHashes) > 0 { + for _, b := range m.ForkIdHashes { + l = len(b) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + l = len(m.ClientId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Capabilities) > 0 { + for _, s := range m.Capabilities { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *CoordinateExecutionNodeRecordsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NodeRecords) > 0 { + for _, s := range m.NodeRecords { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.RetryDelay != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.RetryDelay)) + } + n += len(m.unknownFields) + return n +} + +func (m *ConsensusNodeStatus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NodeRecord) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Name) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ForkDigest) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.NextForkDigest) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FinalizedRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.FinalizedEpoch) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedEpochStartDateTime != nil { + l = (*timestamppb.Timestamp)(m.FinalizedEpochStartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.HeadRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.HeadSlot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HeadSlotStartDateTime != nil { + l = (*timestamppb.Timestamp)(m.HeadSlotStartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Cgc) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.NetworkId != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.NetworkId)) + } + l = len(m.NodeId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.PeerId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ListStalledConsensusNodeRecordsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PageSize != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.PageSize)) + } + n += len(m.unknownFields) + return n +} + +func (m *ListStalledConsensusNodeRecordsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NodeRecords) > 0 { + for _, s := range m.NodeRecords { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *CreateConsensusNodeRecordStatusRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Status != nil { + l = m.Status.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CreateConsensusNodeRecordStatusResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *CreateConsensusNodeRecordStatusesRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Statuses) > 0 { + for _, e := range m.Statuses { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *CreateConsensusNodeRecordStatusesResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *CoordinateConsensusNodeRecordsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NodeRecords) > 0 { + for _, e := range m.NodeRecords { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.Limit != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Limit)) + } + if len(m.NetworkIds) > 0 { + l = 0 + for _, e := range m.NetworkIds { + l += protohelpers.SizeOfVarint(uint64(e)) + } + n += 1 + protohelpers.SizeOfVarint(uint64(l)) + l + } + if len(m.ForkIdHashes) > 0 { + for _, b := range m.ForkIdHashes { + l = len(b) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + l = len(m.ClientId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Capabilities) > 0 { + for _, s := range m.Capabilities { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *CoordinateConsensusNodeRecordsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NodeRecords) > 0 { + for _, s := range m.NodeRecords { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.RetryDelay != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.RetryDelay)) + } + n += len(m.unknownFields) + return n +} + +func (m *GetDiscoveryNodeRecordRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NetworkIds) > 0 { + l = 0 + for _, e := range m.NetworkIds { + l += protohelpers.SizeOfVarint(uint64(e)) + } + n += 1 + protohelpers.SizeOfVarint(uint64(l)) + l + } + if len(m.ForkIdHashes) > 0 { + for _, b := range m.ForkIdHashes { + l = len(b) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *GetDiscoveryNodeRecordResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NodeRecord) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *GetDiscoveryExecutionNodeRecordRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NetworkIds) > 0 { + l = 0 + for _, e := range m.NetworkIds { + l += protohelpers.SizeOfVarint(uint64(e)) + } + n += 1 + protohelpers.SizeOfVarint(uint64(l)) + l + } + if len(m.ForkIdHashes) > 0 { + for _, b := range m.ForkIdHashes { + l = len(b) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *GetDiscoveryExecutionNodeRecordResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NodeRecord) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *GetDiscoveryConsensusNodeRecordRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.NetworkIds) > 0 { + l = 0 + for _, e := range m.NetworkIds { + l += protohelpers.SizeOfVarint(uint64(e)) + } + n += 1 + protohelpers.SizeOfVarint(uint64(l)) + l + } + if len(m.ForkDigests) > 0 { + for _, b := range m.ForkDigests { + l = len(b) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *GetDiscoveryConsensusNodeRecordResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NodeRecord) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BackfillingCheckpointMarker) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.FinalizedEpoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.FinalizedEpoch)) + } + if m.BackfillEpoch != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.BackfillEpoch)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockVoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockProposerSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockDeposit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockAttesterSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockBlsToExecutionChange) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockExecutionTransaction) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockWithdrawal) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV1BeaconBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV1BeaconProposerDuty) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockElaboratedAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV1BeaconValidators) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV1BeaconCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV1BeaconSyncCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocationEthV2BeaconBlockSyncAggregate) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BackfillingCheckpointMarker != nil { + l = m.BackfillingCheckpointMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NetworkId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Type != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Type)) + } + if vtmsg, ok := m.Data.(interface{ SizeVT() int }); ok { + n += vtmsg.SizeVT() + } + n += len(m.unknownFields) + return n +} + +func (m *CannonLocation_EthV2BeaconBlockVoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockVoluntaryExit != nil { + l = m.EthV2BeaconBlockVoluntaryExit.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlockProposerSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockProposerSlashing != nil { + l = m.EthV2BeaconBlockProposerSlashing.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlockDeposit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockDeposit != nil { + l = m.EthV2BeaconBlockDeposit.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlockAttesterSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockAttesterSlashing != nil { + l = m.EthV2BeaconBlockAttesterSlashing.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlockBlsToExecutionChange) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockBlsToExecutionChange != nil { + l = m.EthV2BeaconBlockBlsToExecutionChange.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlockExecutionTransaction) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockExecutionTransaction != nil { + l = m.EthV2BeaconBlockExecutionTransaction.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlockWithdrawal) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockWithdrawal != nil { + l = m.EthV2BeaconBlockWithdrawal.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlock != nil { + l = m.EthV2BeaconBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV1BeaconBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconBlobSidecar != nil { + l = m.EthV1BeaconBlobSidecar.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV1BeaconProposerDuty) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconProposerDuty != nil { + l = m.EthV1BeaconProposerDuty.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlockElaboratedAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockElaboratedAttestation != nil { + l = m.EthV2BeaconBlockElaboratedAttestation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV1BeaconValidators) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconValidators != nil { + l = m.EthV1BeaconValidators.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV1BeaconCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconCommittee != nil { + l = m.EthV1BeaconCommittee.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV1BeaconSyncCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconSyncCommittee != nil { + l = m.EthV1BeaconSyncCommittee.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CannonLocation_EthV2BeaconBlockSyncAggregate) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockSyncAggregate != nil { + l = m.EthV2BeaconBlockSyncAggregate.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *GetCannonLocationRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NetworkId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Type != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Type)) + } + n += len(m.unknownFields) + return n +} + +func (m *GetCannonLocationResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Location != nil { + l = m.Location.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *UpsertCannonLocationRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Location != nil { + l = m.Location.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *UpsertCannonLocationResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *RelayMonitorSlotMarker) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CurrentSlot != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.CurrentSlot)) + } + n += len(m.unknownFields) + return n +} + +func (m *RelayMonitorLocationBidTrace) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SlotMarker != nil { + l = m.SlotMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *RelayMonitorLocationPayloadDelivered) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SlotMarker != nil { + l = m.SlotMarker.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *RelayMonitorLocation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.MetaNetworkName) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.MetaClientName) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.RelayName) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Type != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Type)) + } + if vtmsg, ok := m.Data.(interface{ SizeVT() int }); ok { + n += vtmsg.SizeVT() + } + n += len(m.unknownFields) + return n +} + +func (m *RelayMonitorLocation_BidTrace) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BidTrace != nil { + l = m.BidTrace.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *RelayMonitorLocation_PayloadDelivered) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PayloadDelivered != nil { + l = m.PayloadDelivered.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *GetRelayMonitorLocationRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.MetaNetworkName) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.MetaClientName) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.RelayName) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Type != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Type)) + } + n += len(m.unknownFields) + return n +} + +func (m *GetRelayMonitorLocationResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Location != nil { + l = m.Location.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *UpsertRelayMonitorLocationRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Location != nil { + l = m.Location.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *UpsertRelayMonitorLocationResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *CreateNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateNodeRecordsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateNodeRecordsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecords", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecords = append(m.NodeRecords, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateNodeRecordsResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateNodeRecordsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateNodeRecordsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ListStalledExecutionNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ListStalledExecutionNodeRecordsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ListStalledExecutionNodeRecordsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PageSize", wireType) + } + m.PageSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PageSize |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ListStalledExecutionNodeRecordsResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ListStalledExecutionNodeRecordsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ListStalledExecutionNodeRecordsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecords", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecords = append(m.NodeRecords, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionNodeStatus_Capability) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionNodeStatus_Capability: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionNodeStatus_Capability: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + m.Version = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Version |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionNodeStatus_ForkID) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionNodeStatus_ForkID: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionNodeStatus_ForkID: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hash = append(m.Hash[:0], dAtA[iNdEx:postIndex]...) + if m.Hash == nil { + m.Hash = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Next", wireType) + } + m.Next = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Next |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionNodeStatus) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionNodeStatus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionNodeStatus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecord", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecord = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Capabilities", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Capabilities = append(m.Capabilities, &ExecutionNodeStatus_Capability{}) + if err := m.Capabilities[len(m.Capabilities)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProtocolVersion", wireType) + } + m.ProtocolVersion = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProtocolVersion |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkId", wireType) + } + m.NetworkId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NetworkId |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalDifficulty", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TotalDifficulty = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Head", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Head = append(m.Head[:0], dAtA[iNdEx:postIndex]...) + if m.Head == nil { + m.Head = []byte{} + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Genesis", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Genesis = append(m.Genesis[:0], dAtA[iNdEx:postIndex]...) + if m.Genesis == nil { + m.Genesis = []byte{} + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ForkId == nil { + m.ForkId = &ExecutionNodeStatus_ForkID{} + } + if err := m.ForkId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateExecutionNodeRecordStatusRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateExecutionNodeRecordStatusRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateExecutionNodeRecordStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Status == nil { + m.Status = &ExecutionNodeStatus{} + } + if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateExecutionNodeRecordStatusResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateExecutionNodeRecordStatusResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateExecutionNodeRecordStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CoordinatedNodeRecord) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CoordinatedNodeRecord: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CoordinatedNodeRecord: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecord", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecord = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Connected", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Connected = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectionAttempts", wireType) + } + m.ConnectionAttempts = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ConnectionAttempts |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CoordinateExecutionNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CoordinateExecutionNodeRecordsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CoordinateExecutionNodeRecordsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecords", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecords = append(m.NodeRecords, &CoordinatedNodeRecord{}) + if err := m.NodeRecords[len(m.NodeRecords)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) + } + m.Limit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Limit |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NetworkIds) == 0 { + m.NetworkIds = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkIds", wireType) + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkIdHashes", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ForkIdHashes = append(m.ForkIdHashes, make([]byte, postIndex-iNdEx)) + copy(m.ForkIdHashes[len(m.ForkIdHashes)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClientId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClientId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Capabilities", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Capabilities = append(m.Capabilities, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CoordinateExecutionNodeRecordsResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CoordinateExecutionNodeRecordsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CoordinateExecutionNodeRecordsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecords", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecords = append(m.NodeRecords, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RetryDelay", wireType) + } + m.RetryDelay = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RetryDelay |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ConsensusNodeStatus) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ConsensusNodeStatus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ConsensusNodeStatus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecord", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecord = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkDigest", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ForkDigest = append(m.ForkDigest[:0], dAtA[iNdEx:postIndex]...) + if m.ForkDigest == nil { + m.ForkDigest = []byte{} + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NextForkDigest", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NextForkDigest = append(m.NextForkDigest[:0], dAtA[iNdEx:postIndex]...) + if m.NextForkDigest == nil { + m.NextForkDigest = []byte{} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedRoot", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FinalizedRoot = append(m.FinalizedRoot[:0], dAtA[iNdEx:postIndex]...) + if m.FinalizedRoot == nil { + m.FinalizedRoot = []byte{} + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpoch", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FinalizedEpoch = append(m.FinalizedEpoch[:0], dAtA[iNdEx:postIndex]...) + if m.FinalizedEpoch == nil { + m.FinalizedEpoch = []byte{} + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpochStartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedEpochStartDateTime == nil { + m.FinalizedEpochStartDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.FinalizedEpochStartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadRoot", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.HeadRoot = append(m.HeadRoot[:0], dAtA[iNdEx:postIndex]...) + if m.HeadRoot == nil { + m.HeadRoot = []byte{} + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadSlot", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.HeadSlot = append(m.HeadSlot[:0], dAtA[iNdEx:postIndex]...) + if m.HeadSlot == nil { + m.HeadSlot = []byte{} + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadSlotStartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HeadSlotStartDateTime == nil { + m.HeadSlotStartDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.HeadSlotStartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cgc", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Cgc = append(m.Cgc[:0], dAtA[iNdEx:postIndex]...) + if m.Cgc == nil { + m.Cgc = []byte{} + } + iNdEx = postIndex + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkId", wireType) + } + m.NetworkId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NetworkId |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeerId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PeerId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ListStalledConsensusNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ListStalledConsensusNodeRecordsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ListStalledConsensusNodeRecordsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PageSize", wireType) + } + m.PageSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PageSize |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ListStalledConsensusNodeRecordsResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ListStalledConsensusNodeRecordsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ListStalledConsensusNodeRecordsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecords", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecords = append(m.NodeRecords, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateConsensusNodeRecordStatusRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateConsensusNodeRecordStatusRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateConsensusNodeRecordStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Status == nil { + m.Status = &ConsensusNodeStatus{} + } + if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateConsensusNodeRecordStatusResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateConsensusNodeRecordStatusResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateConsensusNodeRecordStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateConsensusNodeRecordStatusesRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateConsensusNodeRecordStatusesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateConsensusNodeRecordStatusesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Statuses", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Statuses = append(m.Statuses, &ConsensusNodeStatus{}) + if err := m.Statuses[len(m.Statuses)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateConsensusNodeRecordStatusesResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateConsensusNodeRecordStatusesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateConsensusNodeRecordStatusesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CoordinateConsensusNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CoordinateConsensusNodeRecordsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CoordinateConsensusNodeRecordsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecords", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecords = append(m.NodeRecords, &CoordinatedNodeRecord{}) + if err := m.NodeRecords[len(m.NodeRecords)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) + } + m.Limit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Limit |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NetworkIds) == 0 { + m.NetworkIds = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkIds", wireType) + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkIdHashes", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ForkIdHashes = append(m.ForkIdHashes, make([]byte, postIndex-iNdEx)) + copy(m.ForkIdHashes[len(m.ForkIdHashes)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClientId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClientId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Capabilities", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Capabilities = append(m.Capabilities, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CoordinateConsensusNodeRecordsResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CoordinateConsensusNodeRecordsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CoordinateConsensusNodeRecordsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecords", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecords = append(m.NodeRecords, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RetryDelay", wireType) + } + m.RetryDelay = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RetryDelay |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetDiscoveryNodeRecordRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetDiscoveryNodeRecordRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetDiscoveryNodeRecordRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NetworkIds) == 0 { + m.NetworkIds = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkIds", wireType) + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkIdHashes", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ForkIdHashes = append(m.ForkIdHashes, make([]byte, postIndex-iNdEx)) + copy(m.ForkIdHashes[len(m.ForkIdHashes)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetDiscoveryNodeRecordResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetDiscoveryNodeRecordResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetDiscoveryNodeRecordResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecord", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecord = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetDiscoveryExecutionNodeRecordRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetDiscoveryExecutionNodeRecordRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetDiscoveryExecutionNodeRecordRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NetworkIds) == 0 { + m.NetworkIds = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkIds", wireType) + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkIdHashes", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ForkIdHashes = append(m.ForkIdHashes, make([]byte, postIndex-iNdEx)) + copy(m.ForkIdHashes[len(m.ForkIdHashes)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetDiscoveryExecutionNodeRecordResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetDiscoveryExecutionNodeRecordResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetDiscoveryExecutionNodeRecordResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecord", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecord = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetDiscoveryConsensusNodeRecordRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetDiscoveryConsensusNodeRecordRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetDiscoveryConsensusNodeRecordRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NetworkIds) == 0 { + m.NetworkIds = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetworkIds = append(m.NetworkIds, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkIds", wireType) + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkDigests", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ForkDigests = append(m.ForkDigests, make([]byte, postIndex-iNdEx)) + copy(m.ForkDigests[len(m.ForkDigests)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetDiscoveryConsensusNodeRecordResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetDiscoveryConsensusNodeRecordResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetDiscoveryConsensusNodeRecordResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecord", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NodeRecord = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BackfillingCheckpointMarker) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BackfillingCheckpointMarker: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BackfillingCheckpointMarker: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpoch", wireType) + } + m.FinalizedEpoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FinalizedEpoch |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillEpoch", wireType) + } + m.BackfillEpoch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BackfillEpoch |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockVoluntaryExit) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockVoluntaryExit: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockVoluntaryExit: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockProposerSlashing) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockProposerSlashing: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockProposerSlashing: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockDeposit) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockDeposit: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockDeposit: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockAttesterSlashing) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockAttesterSlashing: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockAttesterSlashing: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockBlsToExecutionChange) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockBlsToExecutionChange: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockBlsToExecutionChange: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockExecutionTransaction) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockExecutionTransaction: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockExecutionTransaction: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockWithdrawal) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockWithdrawal: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockWithdrawal: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlock) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlock: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlock: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV1BeaconBlobSidecar) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconBlobSidecar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconBlobSidecar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV1BeaconProposerDuty) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconProposerDuty: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconProposerDuty: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockElaboratedAttestation) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockElaboratedAttestation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockElaboratedAttestation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV1BeaconValidators) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconValidators: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconValidators: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV1BeaconCommittee) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconCommittee: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconCommittee: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV1BeaconSyncCommittee) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconSyncCommittee: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV1BeaconSyncCommittee: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocationEthV2BeaconBlockSyncAggregate) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockSyncAggregate: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocationEthV2BeaconBlockSyncAggregate: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackfillingCheckpointMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BackfillingCheckpointMarker == nil { + m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + } + if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CannonLocation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CannonLocation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NetworkId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= CannonType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockVoluntaryExit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockVoluntaryExit); ok { + if err := oneof.EthV2BeaconBlockVoluntaryExit.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockVoluntaryExit{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockVoluntaryExit{EthV2BeaconBlockVoluntaryExit: v} + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockProposerSlashing", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockProposerSlashing); ok { + if err := oneof.EthV2BeaconBlockProposerSlashing.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockProposerSlashing{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockProposerSlashing{EthV2BeaconBlockProposerSlashing: v} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockDeposit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockDeposit); ok { + if err := oneof.EthV2BeaconBlockDeposit.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockDeposit{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockDeposit{EthV2BeaconBlockDeposit: v} + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockAttesterSlashing", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockAttesterSlashing); ok { + if err := oneof.EthV2BeaconBlockAttesterSlashing.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockAttesterSlashing{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockAttesterSlashing{EthV2BeaconBlockAttesterSlashing: v} + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockBlsToExecutionChange", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockBlsToExecutionChange); ok { + if err := oneof.EthV2BeaconBlockBlsToExecutionChange.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockBlsToExecutionChange{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockBlsToExecutionChange{EthV2BeaconBlockBlsToExecutionChange: v} + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockExecutionTransaction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockExecutionTransaction); ok { + if err := oneof.EthV2BeaconBlockExecutionTransaction.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockExecutionTransaction{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockExecutionTransaction{EthV2BeaconBlockExecutionTransaction: v} + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockWithdrawal", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockWithdrawal); ok { + if err := oneof.EthV2BeaconBlockWithdrawal.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockWithdrawal{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockWithdrawal{EthV2BeaconBlockWithdrawal: v} + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlock); ok { + if err := oneof.EthV2BeaconBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlock{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlock{EthV2BeaconBlock: v} + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconBlobSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconBlobSidecar); ok { + if err := oneof.EthV1BeaconBlobSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV1BeaconBlobSidecar{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV1BeaconBlobSidecar{EthV1BeaconBlobSidecar: v} + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconProposerDuty", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconProposerDuty); ok { + if err := oneof.EthV1BeaconProposerDuty.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV1BeaconProposerDuty{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV1BeaconProposerDuty{EthV1BeaconProposerDuty: v} + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockElaboratedAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockElaboratedAttestation); ok { + if err := oneof.EthV2BeaconBlockElaboratedAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockElaboratedAttestation{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockElaboratedAttestation{EthV2BeaconBlockElaboratedAttestation: v} + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconValidators", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconValidators); ok { + if err := oneof.EthV1BeaconValidators.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV1BeaconValidators{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV1BeaconValidators{EthV1BeaconValidators: v} + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconCommittee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconCommittee); ok { + if err := oneof.EthV1BeaconCommittee.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV1BeaconCommittee{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV1BeaconCommittee{EthV1BeaconCommittee: v} + } + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconSyncCommittee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconSyncCommittee); ok { + if err := oneof.EthV1BeaconSyncCommittee.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV1BeaconSyncCommittee{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV1BeaconSyncCommittee{EthV1BeaconSyncCommittee: v} + } + iNdEx = postIndex + case 18: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockSyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockSyncAggregate); ok { + if err := oneof.EthV2BeaconBlockSyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &CannonLocationEthV2BeaconBlockSyncAggregate{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &CannonLocation_EthV2BeaconBlockSyncAggregate{EthV2BeaconBlockSyncAggregate: v} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetCannonLocationRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetCannonLocationRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetCannonLocationRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NetworkId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= CannonType(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetCannonLocationResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetCannonLocationResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetCannonLocationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Location", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Location == nil { + m.Location = &CannonLocation{} + } + if err := m.Location.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *UpsertCannonLocationRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UpsertCannonLocationRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpsertCannonLocationRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Location", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Location == nil { + m.Location = &CannonLocation{} + } + if err := m.Location.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *UpsertCannonLocationResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UpsertCannonLocationResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpsertCannonLocationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RelayMonitorSlotMarker) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RelayMonitorSlotMarker: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RelayMonitorSlotMarker: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CurrentSlot", wireType) + } + m.CurrentSlot = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CurrentSlot |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RelayMonitorLocationBidTrace) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RelayMonitorLocationBidTrace: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RelayMonitorLocationBidTrace: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SlotMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SlotMarker == nil { + m.SlotMarker = &RelayMonitorSlotMarker{} + } + if err := m.SlotMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RelayMonitorLocationPayloadDelivered) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RelayMonitorLocationPayloadDelivered: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RelayMonitorLocationPayloadDelivered: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SlotMarker", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SlotMarker == nil { + m.SlotMarker = &RelayMonitorSlotMarker{} + } + if err := m.SlotMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RelayMonitorLocation) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RelayMonitorLocation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RelayMonitorLocation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MetaNetworkName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MetaNetworkName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MetaClientName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MetaClientName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RelayName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RelayName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= RelayMonitorType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BidTrace", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*RelayMonitorLocation_BidTrace); ok { + if err := oneof.BidTrace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &RelayMonitorLocationBidTrace{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &RelayMonitorLocation_BidTrace{BidTrace: v} + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PayloadDelivered", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*RelayMonitorLocation_PayloadDelivered); ok { + if err := oneof.PayloadDelivered.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &RelayMonitorLocationPayloadDelivered{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &RelayMonitorLocation_PayloadDelivered{PayloadDelivered: v} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetRelayMonitorLocationRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetRelayMonitorLocationRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetRelayMonitorLocationRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MetaNetworkName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MetaNetworkName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MetaClientName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MetaClientName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RelayName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RelayName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= RelayMonitorType(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetRelayMonitorLocationResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetRelayMonitorLocationResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetRelayMonitorLocationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Location", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Location == nil { + m.Location = &RelayMonitorLocation{} + } + if err := m.Location.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *UpsertRelayMonitorLocationRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UpsertRelayMonitorLocationRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpsertRelayMonitorLocationRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Location", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Location == nil { + m.Location = &RelayMonitorLocation{} + } + if err := m.Location.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *UpsertRelayMonitorLocationResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UpsertRelayMonitorLocationResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpsertRelayMonitorLocationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} diff --git a/pkg/proto/xatu/event_ingester_vtproto.pb.go b/pkg/proto/xatu/event_ingester_vtproto.pb.go new file mode 100644 index 00000000..189cf394 --- /dev/null +++ b/pkg/proto/xatu/event_ingester_vtproto.pb.go @@ -0,0 +1,48622 @@ +// Code generated by protoc-gen-go-vtproto. DO NOT EDIT. +// protoc-gen-go-vtproto version: v0.6.0 +// source: pkg/proto/xatu/event_ingester.proto + +package xatu + +import ( + binary "encoding/binary" + fmt "fmt" + v1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + v2 "github.com/ethpandaops/xatu/pkg/proto/eth/v2" + libp2p "github.com/ethpandaops/xatu/pkg/proto/libp2p" + gossipsub "github.com/ethpandaops/xatu/pkg/proto/libp2p/gossipsub" + mevrelay "github.com/ethpandaops/xatu/pkg/proto/mevrelay" + noderecord "github.com/ethpandaops/xatu/pkg/proto/noderecord" + protohelpers "github.com/planetscale/vtprotobuf/protohelpers" + timestamppb "github.com/planetscale/vtprotobuf/types/known/timestamppb" + wrapperspb "github.com/planetscale/vtprotobuf/types/known/wrapperspb" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" + wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" + io "io" + math "math" +) + +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) +) + +func (m *CreateEventsRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateEventsRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateEventsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Events) > 0 { + for iNdEx := len(m.Events) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Events[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *CreateEventsResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CreateEventsResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *CreateEventsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.EventsIngested != nil { + size, err := (*wrapperspb.UInt64Value)(m.EventsIngested).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Epoch) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Epoch) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Epoch) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.StartDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.StartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Number != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Number)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *EpochV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *EpochV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *EpochV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.StartDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.StartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Number != nil { + size, err := (*wrapperspb.UInt64Value)(m.Number).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Slot) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Slot) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Slot) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.StartDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.StartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Number != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Number)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SlotV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SlotV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SlotV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.StartDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.StartDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Number != nil { + size, err := (*wrapperspb.UInt64Value)(m.Number).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ForkID) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ForkID) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ForkID) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Next) > 0 { + i -= len(m.Next) + copy(dAtA[i:], m.Next) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Next))) + i-- + dAtA[i] = 0x12 + } + if len(m.Hash) > 0 { + i -= len(m.Hash) + copy(dAtA[i:], m.Hash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Hash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Propagation) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Propagation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Propagation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SlotStartDiff != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.SlotStartDiff)) + i-- + dAtA[i] = 0x18 + } + return len(dAtA) - i, nil +} + +func (m *PropagationV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *PropagationV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *PropagationV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SlotStartDiff != nil { + size, err := (*wrapperspb.UInt64Value)(m.SlotStartDiff).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *AttestingValidator) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AttestingValidator) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AttestingValidator) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Index != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Index)) + i-- + dAtA[i] = 0x10 + } + if m.CommitteeIndex != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.CommitteeIndex)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *AttestingValidatorV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AttestingValidatorV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *AttestingValidatorV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Index != nil { + size, err := (*wrapperspb.UInt64Value)(m.Index).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.CommitteeIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.CommitteeIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DebugForkChoiceReorg) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DebugForkChoiceReorg) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DebugForkChoiceReorg) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Event != nil { + size, err := m.Event.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.After != nil { + size, err := m.After.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Before != nil { + size, err := m.Before.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DebugForkChoiceReorgV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DebugForkChoiceReorgV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DebugForkChoiceReorgV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Event != nil { + size, err := m.Event.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.After != nil { + size, err := m.After.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Before != nil { + size, err := m.Before.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Validators) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Validators) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Validators) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Validators) > 0 { + for iNdEx := len(m.Validators) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Validators[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SyncCommitteeData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncCommitteeData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SyncCommitteeData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SyncCommittee != nil { + size, err := m.SyncCommittee.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SyncAggregateData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncAggregateData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *SyncAggregateData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ParticipationCount != nil { + size, err := (*wrapperspb.UInt64Value)(m.ParticipationCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.ValidatorsMissed) > 0 { + for iNdEx := len(m.ValidatorsMissed) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorsMissed[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if len(m.ValidatorsParticipated) > 0 { + for iNdEx := len(m.ValidatorsParticipated) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorsParticipated[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if len(m.SyncCommitteeSignature) > 0 { + i -= len(m.SyncCommitteeSignature) + copy(dAtA[i:], m.SyncCommitteeSignature) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.SyncCommitteeSignature))) + i-- + dAtA[i] = 0x12 + } + if len(m.SyncCommitteeBits) > 0 { + i -= len(m.SyncCommitteeBits) + copy(dAtA[i:], m.SyncCommitteeBits) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.SyncCommitteeBits))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *BlockIdentifier) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlockIdentifier) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *BlockIdentifier) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Root) > 0 { + i -= len(m.Root) + copy(dAtA[i:], m.Root) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Root))) + i-- + dAtA[i] = 0x22 + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionStateSize) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionStateSize) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionStateSize) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Storages) > 0 { + i -= len(m.Storages) + copy(dAtA[i:], m.Storages) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Storages))) + i-- + dAtA[i] = 0x62 + } + if len(m.StorageTrienodes) > 0 { + i -= len(m.StorageTrienodes) + copy(dAtA[i:], m.StorageTrienodes) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StorageTrienodes))) + i-- + dAtA[i] = 0x5a + } + if len(m.StorageTrienodeBytes) > 0 { + i -= len(m.StorageTrienodeBytes) + copy(dAtA[i:], m.StorageTrienodeBytes) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StorageTrienodeBytes))) + i-- + dAtA[i] = 0x52 + } + if len(m.StorageBytes) > 0 { + i -= len(m.StorageBytes) + copy(dAtA[i:], m.StorageBytes) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StorageBytes))) + i-- + dAtA[i] = 0x4a + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x42 + } + if len(m.ContractCodes) > 0 { + i -= len(m.ContractCodes) + copy(dAtA[i:], m.ContractCodes) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ContractCodes))) + i-- + dAtA[i] = 0x3a + } + if len(m.ContractCodeBytes) > 0 { + i -= len(m.ContractCodeBytes) + copy(dAtA[i:], m.ContractCodeBytes) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ContractCodeBytes))) + i-- + dAtA[i] = 0x32 + } + if len(m.BlockNumber) > 0 { + i -= len(m.BlockNumber) + copy(dAtA[i:], m.BlockNumber) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockNumber))) + i-- + dAtA[i] = 0x2a + } + if len(m.Accounts) > 0 { + i -= len(m.Accounts) + copy(dAtA[i:], m.Accounts) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Accounts))) + i-- + dAtA[i] = 0x22 + } + if len(m.AccountTrienodes) > 0 { + i -= len(m.AccountTrienodes) + copy(dAtA[i:], m.AccountTrienodes) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.AccountTrienodes))) + i-- + dAtA[i] = 0x1a + } + if len(m.AccountTrienodeBytes) > 0 { + i -= len(m.AccountTrienodeBytes) + copy(dAtA[i:], m.AccountTrienodeBytes) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.AccountTrienodeBytes))) + i-- + dAtA[i] = 0x12 + } + if len(m.AccountBytes) > 0 { + i -= len(m.AccountBytes) + copy(dAtA[i:], m.AccountBytes) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.AccountBytes))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ConsensusEngineAPINewPayload) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ConsensusEngineAPINewPayload) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ConsensusEngineAPINewPayload) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.MethodVersion) > 0 { + i -= len(m.MethodVersion) + copy(dAtA[i:], m.MethodVersion) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MethodVersion))) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + if len(m.ValidationError) > 0 { + i -= len(m.ValidationError) + copy(dAtA[i:], m.ValidationError) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ValidationError))) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if len(m.LatestValidHash) > 0 { + i -= len(m.LatestValidHash) + copy(dAtA[i:], m.LatestValidHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LatestValidHash))) + i-- + dAtA[i] = 0x7a + } + if len(m.Status) > 0 { + i -= len(m.Status) + copy(dAtA[i:], m.Status) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Status))) + i-- + dAtA[i] = 0x72 + } + if m.BlobCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.BlobCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if m.TxCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.TxCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0x4a + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x42 + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.ProposerIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ProposerIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if len(m.ParentBlockRoot) > 0 { + i -= len(m.ParentBlockRoot) + copy(dAtA[i:], m.ParentBlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentBlockRoot))) + i-- + dAtA[i] = 0x2a + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x22 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.DurationMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.DurationMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RequestedAt != nil { + size, err := (*timestamppb.Timestamp)(m.RequestedAt).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ConsensusEngineAPIGetBlobs) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ConsensusEngineAPIGetBlobs) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ConsensusEngineAPIGetBlobs) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.MethodVersion) > 0 { + i -= len(m.MethodVersion) + copy(dAtA[i:], m.MethodVersion) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MethodVersion))) + i-- + dAtA[i] = 0x5a + } + if len(m.ErrorMessage) > 0 { + i -= len(m.ErrorMessage) + copy(dAtA[i:], m.ErrorMessage) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ErrorMessage))) + i-- + dAtA[i] = 0x52 + } + if len(m.Status) > 0 { + i -= len(m.Status) + copy(dAtA[i:], m.Status) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Status))) + i-- + dAtA[i] = 0x4a + } + if m.ReturnedCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.ReturnedCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if len(m.VersionedHashes) > 0 { + for iNdEx := len(m.VersionedHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.VersionedHashes[iNdEx]) + copy(dAtA[i:], m.VersionedHashes[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.VersionedHashes[iNdEx]))) + i-- + dAtA[i] = 0x3a + } + } + if m.RequestedCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.RequestedCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if len(m.ParentBlockRoot) > 0 { + i -= len(m.ParentBlockRoot) + copy(dAtA[i:], m.ParentBlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentBlockRoot))) + i-- + dAtA[i] = 0x2a + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x22 + } + if m.Slot != nil { + size, err := (*wrapperspb.UInt64Value)(m.Slot).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.DurationMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.DurationMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RequestedAt != nil { + size, err := (*timestamppb.Timestamp)(m.RequestedAt).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionEngineNewPayload) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionEngineNewPayload) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionEngineNewPayload) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.MethodVersion) > 0 { + i -= len(m.MethodVersion) + copy(dAtA[i:], m.MethodVersion) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MethodVersion))) + i-- + dAtA[i] = 0x72 + } + if len(m.ValidationError) > 0 { + i -= len(m.ValidationError) + copy(dAtA[i:], m.ValidationError) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ValidationError))) + i-- + dAtA[i] = 0x6a + } + if len(m.LatestValidHash) > 0 { + i -= len(m.LatestValidHash) + copy(dAtA[i:], m.LatestValidHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.LatestValidHash))) + i-- + dAtA[i] = 0x62 + } + if len(m.Status) > 0 { + i -= len(m.Status) + copy(dAtA[i:], m.Status) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Status))) + i-- + dAtA[i] = 0x5a + } + if m.BlobCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.BlobCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.TxCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.TxCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.GasLimit != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasLimit).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0x32 + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x2a + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.DurationMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.DurationMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.RequestedAt != nil { + size, err := (*timestamppb.Timestamp)(m.RequestedAt).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Source != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Source)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ExecutionEngineGetBlobs) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionEngineGetBlobs) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionEngineGetBlobs) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.ReturnedBlobIndexes) > 0 { + for iNdEx := len(m.ReturnedBlobIndexes) - 1; iNdEx >= 0; iNdEx-- { + size, err := (*wrapperspb.UInt32Value)(m.ReturnedBlobIndexes[iNdEx]).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + } + if len(m.MethodVersion) > 0 { + i -= len(m.MethodVersion) + copy(dAtA[i:], m.MethodVersion) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MethodVersion))) + i-- + dAtA[i] = 0x4a + } + if len(m.ErrorMessage) > 0 { + i -= len(m.ErrorMessage) + copy(dAtA[i:], m.ErrorMessage) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ErrorMessage))) + i-- + dAtA[i] = 0x42 + } + if len(m.Status) > 0 { + i -= len(m.Status) + copy(dAtA[i:], m.Status) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Status))) + i-- + dAtA[i] = 0x3a + } + if m.ReturnedCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.ReturnedCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if len(m.VersionedHashes) > 0 { + for iNdEx := len(m.VersionedHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.VersionedHashes[iNdEx]) + copy(dAtA[i:], m.VersionedHashes[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.VersionedHashes[iNdEx]))) + i-- + dAtA[i] = 0x2a + } + } + if m.RequestedCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.RequestedCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.DurationMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.DurationMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.RequestedAt != nil { + size, err := (*timestamppb.Timestamp)(m.RequestedAt).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Source != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Source)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_Ethereum_Network) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_Ethereum_Network) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Ethereum_Network) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Id != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Id)) + i-- + dAtA[i] = 0x10 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_Ethereum_Execution) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_Ethereum_Execution) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Ethereum_Execution) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.VersionPatch) > 0 { + i -= len(m.VersionPatch) + copy(dAtA[i:], m.VersionPatch) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.VersionPatch))) + i-- + dAtA[i] = 0x32 + } + if len(m.VersionMinor) > 0 { + i -= len(m.VersionMinor) + copy(dAtA[i:], m.VersionMinor) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.VersionMinor))) + i-- + dAtA[i] = 0x2a + } + if len(m.VersionMajor) > 0 { + i -= len(m.VersionMajor) + copy(dAtA[i:], m.VersionMajor) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.VersionMajor))) + i-- + dAtA[i] = 0x22 + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x1a + } + if len(m.Implementation) > 0 { + i -= len(m.Implementation) + copy(dAtA[i:], m.Implementation) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Implementation))) + i-- + dAtA[i] = 0x12 + } + if m.ForkId != nil { + size, err := m.ForkId.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_Ethereum_Consensus) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_Ethereum_Consensus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Ethereum_Consensus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x12 + } + if len(m.Implementation) > 0 { + i -= len(m.Implementation) + copy(dAtA[i:], m.Implementation) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Implementation))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_Ethereum) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_Ethereum) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Ethereum) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Consensus != nil { + size, err := m.Consensus.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Execution != nil { + size, err := m.Execution.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Network != nil { + size, err := m.Network.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.AttestingValidator != nil { + size, err := m.AttestingValidator.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.AttestingValidator != nil { + size, err := m.AttestingValidator.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Contribution != nil { + size, err := m.Contribution.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Contribution != nil { + size, err := m.Contribution.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_ForkChoiceSnapshot) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_ForkChoiceSnapshot) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_ForkChoiceSnapshot) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Timestamp != nil { + size, err := (*timestamppb.Timestamp)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.RequestDurationMs != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.RequestDurationMs)) + i-- + dAtA[i] = 0x20 + } + if m.RequestedAtSlotStartDiffMs != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.RequestedAtSlotStartDiffMs)) + i-- + dAtA[i] = 0x18 + } + if m.RequestSlot != nil { + size, err := m.RequestSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RequestEpoch != nil { + size, err := m.RequestEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_ForkChoiceSnapshotV2) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_ForkChoiceSnapshotV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_ForkChoiceSnapshotV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Timestamp != nil { + size, err := (*timestamppb.Timestamp)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.RequestDurationMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.RequestDurationMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.RequestedAtSlotStartDiffMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.RequestedAtSlotStartDiffMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.RequestSlot != nil { + size, err := m.RequestSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RequestEpoch != nil { + size, err := m.RequestEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Snapshot != nil { + size, err := m.Snapshot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Snapshot != nil { + size, err := m.Snapshot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.After != nil { + size, err := m.After.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Before != nil { + size, err := m.Before.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.After != nil { + size, err := m.After.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Before != nil { + size, err := m.Before.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.StateId) > 0 { + i -= len(m.StateId) + copy(dAtA[i:], m.StateId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateId))) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SyncCommitteePeriod != nil { + size, err := (*wrapperspb.UInt64Value)(m.SyncCommitteePeriod).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.SyncCommitteePeriod != nil { + size, err := (*wrapperspb.UInt64Value)(m.SyncCommitteePeriod).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalMempoolTransactionData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalMempoolTransactionData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalMempoolTransactionData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.CallDataSize) > 0 { + i -= len(m.CallDataSize) + copy(dAtA[i:], m.CallDataSize) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.CallDataSize))) + i-- + dAtA[i] = 0x4a + } + if len(m.Size) > 0 { + i -= len(m.Size) + copy(dAtA[i:], m.Size) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Size))) + i-- + dAtA[i] = 0x42 + } + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x3a + } + if m.Gas != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Gas)) + i-- + dAtA[i] = 0x30 + } + if len(m.GasPrice) > 0 { + i -= len(m.GasPrice) + copy(dAtA[i:], m.GasPrice) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.GasPrice))) + i-- + dAtA[i] = 0x2a + } + if m.Nonce != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Nonce)) + i-- + dAtA[i] = 0x20 + } + if len(m.To) > 0 { + i -= len(m.To) + copy(dAtA[i:], m.To) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.To))) + i-- + dAtA[i] = 0x1a + } + if len(m.From) > 0 { + i -= len(m.From) + copy(dAtA[i:], m.From) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.From))) + i-- + dAtA[i] = 0x12 + } + if len(m.Hash) > 0 { + i -= len(m.Hash) + copy(dAtA[i:], m.Hash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Hash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalMempoolTransactionV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalMempoolTransactionV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalMempoolTransactionV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlobSidecarsEmptySize) > 0 { + i -= len(m.BlobSidecarsEmptySize) + copy(dAtA[i:], m.BlobSidecarsEmptySize) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobSidecarsEmptySize))) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + if len(m.BlobSidecarsSize) > 0 { + i -= len(m.BlobSidecarsSize) + copy(dAtA[i:], m.BlobSidecarsSize) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobSidecarsSize))) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if len(m.BlobHashes) > 0 { + for iNdEx := len(m.BlobHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.BlobHashes[iNdEx]) + copy(dAtA[i:], m.BlobHashes[iNdEx]) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobHashes[iNdEx]))) + i-- + dAtA[i] = 0x7a + } + } + if len(m.BlobGasFeeCap) > 0 { + i -= len(m.BlobGasFeeCap) + copy(dAtA[i:], m.BlobGasFeeCap) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobGasFeeCap))) + i-- + dAtA[i] = 0x72 + } + if m.BlobGas != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlobGas).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if len(m.GasFeeCap) > 0 { + i -= len(m.GasFeeCap) + copy(dAtA[i:], m.GasFeeCap) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.GasFeeCap))) + i-- + dAtA[i] = 0x62 + } + if len(m.GasTipCap) > 0 { + i -= len(m.GasTipCap) + copy(dAtA[i:], m.GasTipCap) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.GasTipCap))) + i-- + dAtA[i] = 0x5a + } + if m.Type != nil { + size, err := (*wrapperspb.UInt32Value)(m.Type).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if len(m.CallDataSize) > 0 { + i -= len(m.CallDataSize) + copy(dAtA[i:], m.CallDataSize) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.CallDataSize))) + i-- + dAtA[i] = 0x4a + } + if len(m.Size) > 0 { + i -= len(m.Size) + copy(dAtA[i:], m.Size) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Size))) + i-- + dAtA[i] = 0x42 + } + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x3a + } + if m.Gas != nil { + size, err := (*wrapperspb.UInt64Value)(m.Gas).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if len(m.GasPrice) > 0 { + i -= len(m.GasPrice) + copy(dAtA[i:], m.GasPrice) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.GasPrice))) + i-- + dAtA[i] = 0x2a + } + if m.Nonce != nil { + size, err := (*wrapperspb.UInt64Value)(m.Nonce).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if len(m.To) > 0 { + i -= len(m.To) + copy(dAtA[i:], m.To) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.To))) + i-- + dAtA[i] = 0x1a + } + if len(m.From) > 0 { + i -= len(m.From) + copy(dAtA[i:], m.From) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.From))) + i-- + dAtA[i] = 0x12 + } + if len(m.Hash) > 0 { + i -= len(m.Hash) + copy(dAtA[i:], m.Hash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Hash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.TransactionsTotalBytes != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.TransactionsTotalBytes)) + i-- + dAtA[i] = 0x30 + } + if m.TransactionsCount != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.TransactionsCount)) + i-- + dAtA[i] = 0x28 + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.FinalizedWhenRequested { + i-- + if m.FinalizedWhenRequested { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x50 + } + if m.TotalBytesCompressed != nil { + size, err := (*wrapperspb.UInt64Value)(m.TotalBytesCompressed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.TotalBytes != nil { + size, err := (*wrapperspb.UInt64Value)(m.TotalBytes).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.TransactionsTotalBytesCompressed != nil { + size, err := (*wrapperspb.UInt64Value)(m.TransactionsTotalBytesCompressed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.TransactionsTotalBytes != nil { + size, err := (*wrapperspb.UInt64Value)(m.TransactionsTotalBytes).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.TransactionsCount != nil { + size, err := (*wrapperspb.UInt64Value)(m.TransactionsCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.BlockRoot) > 0 { + i -= len(m.BlockRoot) + copy(dAtA[i:], m.BlockRoot) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockRoot))) + i-- + dAtA[i] = 0x22 + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockDepositData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockDepositData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockDepositData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.BlobSidecarsEmptySize) > 0 { + i -= len(m.BlobSidecarsEmptySize) + copy(dAtA[i:], m.BlobSidecarsEmptySize) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobSidecarsEmptySize))) + i-- + dAtA[i] = 0x32 + } + if len(m.BlobSidecarsSize) > 0 { + i -= len(m.BlobSidecarsSize) + copy(dAtA[i:], m.BlobSidecarsSize) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlobSidecarsSize))) + i-- + dAtA[i] = 0x2a + } + if len(m.CallDataSize) > 0 { + i -= len(m.CallDataSize) + copy(dAtA[i:], m.CallDataSize) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.CallDataSize))) + i-- + dAtA[i] = 0x22 + } + if len(m.Size) > 0 { + i -= len(m.Size) + copy(dAtA[i:], m.Size) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Size))) + i-- + dAtA[i] = 0x1a + } + if m.PositionInBlock != nil { + size, err := (*wrapperspb.UInt64Value)(m.PositionInBlock).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AttestationDataSnapshot) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AttestationDataSnapshot) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AttestationDataSnapshot) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Timestamp != nil { + size, err := (*timestamppb.Timestamp)(m.Timestamp).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.RequestDurationMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.RequestDurationMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.RequestedAtSlotStartDiffMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.RequestedAtSlotStartDiffMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Snapshot != nil { + size, err := m.Snapshot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.DataEmptySize != nil { + size, err := (*wrapperspb.UInt64Value)(m.DataEmptySize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if len(m.VersionedHash) > 0 { + i -= len(m.VersionedHash) + copy(dAtA[i:], m.VersionedHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.VersionedHash))) + i-- + dAtA[i] = 0x22 + } + if m.DataSize != nil { + size, err := (*wrapperspb.UInt64Value)(m.DataSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalBeaconP2PAttestationData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalBeaconP2PAttestationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalBeaconP2PAttestationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Validated != nil { + size, err := (*wrapperspb.BoolValue)(m.Validated).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.Subnet != nil { + size, err := (*wrapperspb.UInt32Value)(m.Subnet).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.Peer != nil { + size, err := m.Peer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.AttestingValidator != nil { + size, err := m.AttestingValidator.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1ProposerDutyData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1ProposerDutyData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1ProposerDutyData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.StateId) > 0 { + i -= len(m.StateId) + copy(dAtA[i:], m.StateId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.StateId))) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.PositionInBlock != nil { + size, err := (*wrapperspb.UInt64Value)(m.PositionInBlock).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Block != nil { + size, err := m.Block.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceAddPeerData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceAddPeerData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceAddPeerData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRemovePeerData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRemovePeerData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRemovePeerData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRecvRPCData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRecvRPCData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRecvRPCData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceSendRPCData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceSendRPCData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceSendRPCData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceDropRPCData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceDropRPCData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceDropRPCData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceJoinData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceJoinData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceJoinData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceLeaveData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceLeaveData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceLeaveData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGraftData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGraftData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceGraftData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTracePruneData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTracePruneData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTracePruneData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceDeliverMessageData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceDeliverMessageData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceDeliverMessageData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTracePublishMessageData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTracePublishMessageData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTracePublishMessageData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRejectMessageData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRejectMessageData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRejectMessageData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceConnectedData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceConnectedData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceConnectedData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceDisconnectedData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceDisconnectedData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceDisconnectedData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleStatusData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleStatusData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleStatusData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceIdentifyData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceIdentifyData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceIdentifyData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.MessageSize != nil { + size, err := (*wrapperspb.UInt32Value)(m.MessageSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.MessageSize != nil { + size, err := (*wrapperspb.UInt32Value)(m.MessageSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.AttestingValidator != nil { + size, err := m.AttestingValidator.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Source != nil { + size, err := m.Source.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.MessageSize != nil { + size, err := (*wrapperspb.UInt32Value)(m.MessageSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.AggregatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.AggregatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.MessageSize != nil { + size, err := (*wrapperspb.UInt32Value)(m.MessageSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.MessageId != nil { + size, err := (*wrapperspb.StringValue)(m.MessageId).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.MessageSize != nil { + size, err := (*wrapperspb.UInt32Value)(m.MessageSize).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.Topic != nil { + size, err := (*wrapperspb.StringValue)(m.Topic).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.Metadata != nil { + size, err := m.Metadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Propagation != nil { + size, err := m.Propagation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1ValidatorsData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1ValidatorsData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1ValidatorsData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ResponseAtSlotTime != nil { + size, err := (*wrapperspb.UInt64Value)(m.ResponseAtSlotTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.RequestedAtSlotTime != nil { + size, err := (*wrapperspb.UInt64Value)(m.RequestedAtSlotTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Relay != nil { + size, err := m.Relay.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ResponseAtSlotTime != nil { + size, err := (*wrapperspb.UInt64Value)(m.ResponseAtSlotTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.RequestedAtSlotTime != nil { + size, err := (*wrapperspb.UInt64Value)(m.RequestedAtSlotTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Relay != nil { + size, err := m.Relay.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.RequestedAt != nil { + size, err := (*timestamppb.Timestamp)(m.RequestedAt).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.RequestDurationMs != nil { + size, err := (*wrapperspb.UInt64Value)(m.RequestDurationMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if len(m.ConsensusValue) > 0 { + i -= len(m.ConsensusValue) + copy(dAtA[i:], m.ConsensusValue) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ConsensusValue))) + i-- + dAtA[i] = 0x52 + } + if len(m.ExecutionValue) > 0 { + i -= len(m.ExecutionValue) + copy(dAtA[i:], m.ExecutionValue) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ExecutionValue))) + i-- + dAtA[i] = 0x4a + } + if m.TotalBytesCompressed != nil { + size, err := (*wrapperspb.UInt64Value)(m.TotalBytesCompressed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.TotalBytes != nil { + size, err := (*wrapperspb.UInt64Value)(m.TotalBytes).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.TransactionsTotalBytesCompressed != nil { + size, err := (*wrapperspb.UInt64Value)(m.TransactionsTotalBytesCompressed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.TransactionsTotalBytes != nil { + size, err := (*wrapperspb.UInt64Value)(m.TransactionsTotalBytes).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.TransactionsCount != nil { + size, err := (*wrapperspb.UInt64Value)(m.TransactionsCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ValidatorIndex != nil { + size, err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.WallclockEpoch != nil { + size, err := m.WallclockEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.WallclockSlot != nil { + size, err := m.WallclockSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Relay != nil { + size, err := m.Relay.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalNodeRecordConsensusData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalNodeRecordConsensusData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalNodeRecordConsensusData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.HeadEpoch != nil { + size, err := m.HeadEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.HeadSlot != nil { + size, err := m.HeadSlot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.FinalizedEpoch != nil { + size, err := m.FinalizedEpoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Slot != nil { + size, err := m.Slot.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Epoch != nil { + size, err := m.Epoch.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if vtmsg, ok := m.AdditionalData.(interface { + MarshalToSizedBufferVT([]byte) (int, error) + }); ok { + size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + } + if len(m.PresetName) > 0 { + i -= len(m.PresetName) + copy(dAtA[i:], m.PresetName) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.PresetName))) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x82 + } + if m.ModuleName != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ModuleName)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xf8 + } + if len(m.Labels) > 0 { + for k := range m.Labels { + v := m.Labels[k] + baseI := i + i -= len(v) + copy(dAtA[i:], v) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(v))) + i-- + dAtA[i] = 0x12 + i -= len(k) + copy(dAtA[i:], k) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(k))) + i-- + dAtA[i] = 0xa + i = protohelpers.EncodeVarint(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x4a + } + } + if m.Ethereum != nil { + size, err := m.Ethereum.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.ClockDrift != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ClockDrift)) + i-- + dAtA[i] = 0x30 + } + if len(m.Os) > 0 { + i -= len(m.Os) + copy(dAtA[i:], m.Os) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Os))) + i-- + dAtA[i] = 0x2a + } + if len(m.Implementation) > 0 { + i -= len(m.Implementation) + copy(dAtA[i:], m.Implementation) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Implementation))) + i-- + dAtA[i] = 0x22 + } + if len(m.Id) > 0 { + i -= len(m.Id) + copy(dAtA[i:], m.Id) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Id))) + i-- + dAtA[i] = 0x1a + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x12 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientMeta_EthV1EventsAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsAttestation != nil { + size, err := m.EthV1EventsAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsHead) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsHead) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsHead != nil { + size, err := m.EthV1EventsHead.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsBlock != nil { + size, err := m.EthV1EventsBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsVoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsVoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsVoluntaryExit != nil { + size, err := m.EthV1EventsVoluntaryExit.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsFinalizedCheckpoint) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsFinalizedCheckpoint) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsFinalizedCheckpoint != nil { + size, err := m.EthV1EventsFinalizedCheckpoint.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x72 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsChainReorg) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsChainReorg) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsChainReorg != nil { + size, err := m.EthV1EventsChainReorg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsContributionAndProof) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsContributionAndProof) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsContributionAndProof != nil { + size, err := m.EthV1EventsContributionAndProof.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_MempoolTransaction) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_MempoolTransaction) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.MempoolTransaction != nil { + size, err := m.MempoolTransaction.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlock != nil { + size, err := m.EthV2BeaconBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1DebugForkChoice) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1DebugForkChoice) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1DebugForkChoice != nil { + size, err := m.EthV1DebugForkChoice.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x9a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1DebugForkChoiceReorg) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1DebugForkChoiceReorg) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1DebugForkChoiceReorg != nil { + size, err := m.EthV1DebugForkChoiceReorg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1BeaconCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1BeaconCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconCommittee != nil { + size, err := m.EthV1BeaconCommittee.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1ValidatorAttestationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1ValidatorAttestationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1ValidatorAttestationData != nil { + size, err := m.EthV1ValidatorAttestationData.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsAttestationV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsAttestationV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsAttestationV2 != nil { + size, err := m.EthV1EventsAttestationV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsHeadV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsHeadV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsHeadV2 != nil { + size, err := m.EthV1EventsHeadV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsBlockV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsBlockV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsBlockV2 != nil { + size, err := m.EthV1EventsBlockV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsVoluntaryExitV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsVoluntaryExitV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsVoluntaryExitV2 != nil { + size, err := m.EthV1EventsVoluntaryExitV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsFinalizedCheckpointV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsFinalizedCheckpointV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsFinalizedCheckpointV2 != nil { + size, err := m.EthV1EventsFinalizedCheckpointV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsChainReorgV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsChainReorgV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsChainReorgV2 != nil { + size, err := m.EthV1EventsChainReorgV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsContributionAndProofV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsContributionAndProofV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsContributionAndProofV2 != nil { + size, err := m.EthV1EventsContributionAndProofV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_MempoolTransactionV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_MempoolTransactionV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.MempoolTransactionV2 != nil { + size, err := m.MempoolTransactionV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xfa + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockV2 != nil { + size, err := m.EthV2BeaconBlockV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1DebugForkChoiceV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1DebugForkChoiceV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1DebugForkChoiceV2 != nil { + size, err := m.EthV1DebugForkChoiceV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1DebugForkChoiceReorgV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1DebugForkChoiceReorgV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1DebugForkChoiceReorgV2 != nil { + size, err := m.EthV1DebugForkChoiceReorgV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockAttesterSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockAttesterSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockAttesterSlashing != nil { + size, err := m.EthV2BeaconBlockAttesterSlashing.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0x9a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockProposerSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockProposerSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockProposerSlashing != nil { + size, err := m.EthV2BeaconBlockProposerSlashing.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockVoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockVoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockVoluntaryExit != nil { + size, err := m.EthV2BeaconBlockVoluntaryExit.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockDeposit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockDeposit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockDeposit != nil { + size, err := m.EthV2BeaconBlockDeposit.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockBlsToExecutionChange) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockBlsToExecutionChange) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockBlsToExecutionChange != nil { + size, err := m.EthV2BeaconBlockBlsToExecutionChange.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xba + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockExecutionTransaction) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockExecutionTransaction) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockExecutionTransaction != nil { + size, err := m.EthV2BeaconBlockExecutionTransaction.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockWithdrawal) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockWithdrawal) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockWithdrawal != nil { + size, err := m.EthV2BeaconBlockWithdrawal.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsBlobSidecar != nil { + size, err := m.EthV1EventsBlobSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1BeaconBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1BeaconBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconBlobSidecar != nil { + size, err := m.EthV1BeaconBlobSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_BeaconP2PAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_BeaconP2PAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.BeaconP2PAttestation != nil { + size, err := m.BeaconP2PAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1ProposerDuty) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1ProposerDuty) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1ProposerDuty != nil { + size, err := m.EthV1ProposerDuty.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockElaboratedAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockElaboratedAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockElaboratedAttestation != nil { + size, err := m.EthV2BeaconBlockElaboratedAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xfa + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceAddPeer) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceAddPeer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceAddPeer != nil { + size, err := m.Libp2PTraceAddPeer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRemovePeer) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRemovePeer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRemovePeer != nil { + size, err := m.Libp2PTraceRemovePeer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRecvRpc) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRecvRpc) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRecvRpc != nil { + size, err := m.Libp2PTraceRecvRpc.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceSendRpc) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceSendRpc) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceSendRpc != nil { + size, err := m.Libp2PTraceSendRpc.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x9a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceJoin) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceJoin) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceJoin != nil { + size, err := m.Libp2PTraceJoin.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceConnected) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceConnected) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceConnected != nil { + size, err := m.Libp2PTraceConnected.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceDisconnected) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceDisconnected) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceDisconnected != nil { + size, err := m.Libp2PTraceDisconnected.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceHandleMetadata) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceHandleMetadata) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceHandleMetadata != nil { + size, err := m.Libp2PTraceHandleMetadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xba + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceHandleStatus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceHandleStatus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceHandleStatus != nil { + size, err := m.Libp2PTraceHandleStatus.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceGossipsubBeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceGossipsubBeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubBeaconBlock != nil { + size, err := m.Libp2PTraceGossipsubBeaconBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceGossipsubBeaconAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceGossipsubBeaconAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubBeaconAttestation != nil { + size, err := m.Libp2PTraceGossipsubBeaconAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceGossipsubBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceGossipsubBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubBlobSidecar != nil { + size, err := m.Libp2PTraceGossipsubBlobSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1Validators) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1Validators) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1Validators != nil { + size, err := m.EthV1Validators.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_MevRelayBidTraceBuilderBlockSubmission) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_MevRelayBidTraceBuilderBlockSubmission) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.MevRelayBidTraceBuilderBlockSubmission != nil { + size, err := m.MevRelayBidTraceBuilderBlockSubmission.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_MevRelayPayloadDelivered) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_MevRelayPayloadDelivered) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.MevRelayPayloadDelivered != nil { + size, err := m.MevRelayPayloadDelivered.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV3ValidatorBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV3ValidatorBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV3ValidatorBlock != nil { + size, err := m.EthV3ValidatorBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_MevRelayValidatorRegistration) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_MevRelayValidatorRegistration) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.MevRelayValidatorRegistration != nil { + size, err := m.MevRelayValidatorRegistration.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsBlockGossip) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsBlockGossip) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsBlockGossip != nil { + size, err := m.EthV1EventsBlockGossip.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x9a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceDropRpc) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceDropRpc) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceDropRpc != nil { + size, err := m.Libp2PTraceDropRpc.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceLeave) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceLeave) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceLeave != nil { + size, err := m.Libp2PTraceLeave.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceGraft) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceGraft) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGraft != nil { + size, err := m.Libp2PTraceGraft.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTracePrune) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTracePrune) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTracePrune != nil { + size, err := m.Libp2PTracePrune.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xba + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceDuplicateMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceDuplicateMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceDuplicateMessage != nil { + size, err := m.Libp2PTraceDuplicateMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceDeliverMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceDeliverMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceDeliverMessage != nil { + size, err := m.Libp2PTraceDeliverMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTracePublishMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTracePublishMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTracePublishMessage != nil { + size, err := m.Libp2PTracePublishMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRejectMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRejectMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRejectMessage != nil { + size, err := m.Libp2PTraceRejectMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlIhave) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRpcMetaControlIhave) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlIhave != nil { + size, err := m.Libp2PTraceRpcMetaControlIhave.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlIwant) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRpcMetaControlIwant) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlIwant != nil { + size, err := m.Libp2PTraceRpcMetaControlIwant.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlIdontwant) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRpcMetaControlIdontwant) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlIdontwant != nil { + size, err := m.Libp2PTraceRpcMetaControlIdontwant.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlGraft) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRpcMetaControlGraft) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlGraft != nil { + size, err := m.Libp2PTraceRpcMetaControlGraft.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xfa + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlPrune) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRpcMetaControlPrune) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlPrune != nil { + size, err := m.Libp2PTraceRpcMetaControlPrune.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRpcMetaSubscription) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRpcMetaSubscription) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaSubscription != nil { + size, err := m.Libp2PTraceRpcMetaSubscription.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRpcMetaMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRpcMetaMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaMessage != nil { + size, err := m.Libp2PTraceRpcMetaMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_NodeRecordConsensus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_NodeRecordConsensus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.NodeRecordConsensus != nil { + size, err := m.NodeRecordConsensus.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0x9a + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceGossipsubAggregateAndProof) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceGossipsubAggregateAndProof) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubAggregateAndProof != nil { + size, err := m.Libp2PTraceGossipsubAggregateAndProof.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1EventsDataColumnSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1EventsDataColumnSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsDataColumnSidecar != nil { + size, err := m.EthV1EventsDataColumnSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceGossipsubDataColumnSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceGossipsubDataColumnSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubDataColumnSidecar != nil { + size, err := m.Libp2PTraceGossipsubDataColumnSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceSyntheticHeartbeat) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceSyntheticHeartbeat) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceSyntheticHeartbeat != nil { + size, err := m.Libp2PTraceSyntheticHeartbeat.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xba + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceRpcDataColumnCustodyProbe) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceRpcDataColumnCustodyProbe) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcDataColumnCustodyProbe != nil { + size, err := m.Libp2PTraceRpcDataColumnCustodyProbe.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_ConsensusEngineApiNewPayload) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_ConsensusEngineApiNewPayload) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ConsensusEngineApiNewPayload != nil { + size, err := m.ConsensusEngineApiNewPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_ConsensusEngineApiGetBlobs) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_ConsensusEngineApiGetBlobs) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ConsensusEngineApiGetBlobs != nil { + size, err := m.ConsensusEngineApiGetBlobs.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1BeaconBlob) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1BeaconBlob) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconBlob != nil { + size, err := m.EthV1BeaconBlob.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV1BeaconSyncCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV1BeaconSyncCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconSyncCommittee != nil { + size, err := m.EthV1BeaconSyncCommittee.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_EthV2BeaconBlockSyncAggregate) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_EthV2BeaconBlockSyncAggregate) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockSyncAggregate != nil { + size, err := m.EthV2BeaconBlockSyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *ClientMeta_Libp2PTraceIdentify) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ClientMeta_Libp2PTraceIdentify) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceIdentify != nil { + size, err := m.Libp2PTraceIdentify.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *ServerMeta_Event) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_Event) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_Event) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ReceivedDateTime != nil { + size, err := (*timestamppb.Timestamp)(m.ReceivedDateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_Geo) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_Geo) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_Geo) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.AutonomousSystemOrganization) > 0 { + i -= len(m.AutonomousSystemOrganization) + copy(dAtA[i:], m.AutonomousSystemOrganization) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.AutonomousSystemOrganization))) + i-- + dAtA[i] = 0x42 + } + if m.AutonomousSystemNumber != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.AutonomousSystemNumber)) + i-- + dAtA[i] = 0x38 + } + if m.Longitude != 0 { + i -= 8 + binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Longitude)))) + i-- + dAtA[i] = 0x31 + } + if m.Latitude != 0 { + i -= 8 + binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Latitude)))) + i-- + dAtA[i] = 0x29 + } + if len(m.ContinentCode) > 0 { + i -= len(m.ContinentCode) + copy(dAtA[i:], m.ContinentCode) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.ContinentCode))) + i-- + dAtA[i] = 0x22 + } + if len(m.CountryCode) > 0 { + i -= len(m.CountryCode) + copy(dAtA[i:], m.CountryCode) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.CountryCode))) + i-- + dAtA[i] = 0x1a + } + if len(m.Country) > 0 { + i -= len(m.Country) + copy(dAtA[i:], m.Country) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Country))) + i-- + dAtA[i] = 0x12 + } + if len(m.City) > 0 { + i -= len(m.City) + copy(dAtA[i:], m.City) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.City))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_Client) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_Client) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_Client) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.User) > 0 { + i -= len(m.User) + copy(dAtA[i:], m.User) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.User))) + i-- + dAtA[i] = 0x22 + } + if len(m.Group) > 0 { + i -= len(m.Group) + copy(dAtA[i:], m.Group) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Group))) + i-- + dAtA[i] = 0x1a + } + if m.Geo != nil { + size, err := m.Geo.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.IP) > 0 { + i -= len(m.IP) + copy(dAtA[i:], m.IP) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.IP))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_Peer) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_Peer) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_Peer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Geo != nil { + size, err := m.Geo.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_AdditionalBeaconP2PAttestationData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_AdditionalBeaconP2PAttestationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_AdditionalBeaconP2PAttestationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Peer != nil { + size, err := m.Peer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_AdditionalLibp2PTraceConnectedData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_AdditionalLibp2PTraceConnectedData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_AdditionalLibp2PTraceConnectedData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Peer != nil { + size, err := m.Peer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_AdditionalLibp2PTraceDisconnectedData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_AdditionalLibp2PTraceDisconnectedData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_AdditionalLibp2PTraceDisconnectedData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Peer != nil { + size, err := m.Peer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Peer != nil { + size, err := m.Peer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_AdditionalLibp2PTraceIdentifyData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_AdditionalLibp2PTraceIdentifyData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_AdditionalLibp2PTraceIdentifyData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Peer != nil { + size, err := m.Peer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_AdditionalNodeRecordConsensusData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_AdditionalNodeRecordConsensusData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_AdditionalNodeRecordConsensusData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Geo != nil { + size, err := m.Geo.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_AdditionalNodeRecordExecutionData) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta_AdditionalNodeRecordExecutionData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_AdditionalNodeRecordExecutionData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Geo != nil { + size, err := m.Geo.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ServerMeta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if vtmsg, ok := m.AdditionalData.(interface { + MarshalToSizedBufferVT([]byte) (int, error) + }); ok { + size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + } + if m.Client != nil { + size, err := m.Client.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Event != nil { + size, err := m.Event.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ServerMeta_BEACON_P2P_ATTESTATION) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_BEACON_P2P_ATTESTATION) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.BEACON_P2P_ATTESTATION != nil { + size, err := m.BEACON_P2P_ATTESTATION.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *ServerMeta_LIBP2P_TRACE_CONNECTED) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_LIBP2P_TRACE_CONNECTED) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.LIBP2P_TRACE_CONNECTED != nil { + size, err := m.LIBP2P_TRACE_CONNECTED.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + return len(dAtA) - i, nil +} +func (m *ServerMeta_LIBP2P_TRACE_DISCONNECTED) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_LIBP2P_TRACE_DISCONNECTED) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.LIBP2P_TRACE_DISCONNECTED != nil { + size, err := m.LIBP2P_TRACE_DISCONNECTED.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + return len(dAtA) - i, nil +} +func (m *ServerMeta_NODE_RECORD_CONSENSUS) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_NODE_RECORD_CONSENSUS) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.NODE_RECORD_CONSENSUS != nil { + size, err := m.NODE_RECORD_CONSENSUS.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + return len(dAtA) - i, nil +} +func (m *ServerMeta_NODE_RECORD_EXECUTION) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_NODE_RECORD_EXECUTION) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.NODE_RECORD_EXECUTION != nil { + size, err := m.NODE_RECORD_EXECUTION.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + return len(dAtA) - i, nil +} +func (m *ServerMeta_LIBP2P_TRACE_SYNTHETIC_HEARTBEAT) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_LIBP2P_TRACE_SYNTHETIC_HEARTBEAT) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.LIBP2P_TRACE_SYNTHETIC_HEARTBEAT != nil { + size, err := m.LIBP2P_TRACE_SYNTHETIC_HEARTBEAT.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *ServerMeta_LIBP2P_TRACE_IDENTIFY) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ServerMeta_LIBP2P_TRACE_IDENTIFY) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.LIBP2P_TRACE_IDENTIFY != nil { + size, err := m.LIBP2P_TRACE_IDENTIFY.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *Meta) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Meta) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Meta) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Server != nil { + size, err := m.Server.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Client != nil { + size, err := m.Client.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Event) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Event) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *Event) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Id) > 0 { + i -= len(m.Id) + copy(dAtA[i:], m.Id) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Id))) + i-- + dAtA[i] = 0x1a + } + if m.DateTime != nil { + size, err := (*timestamppb.Timestamp)(m.DateTime).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Name != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.Name)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ExecutionBlockMetrics_StateReads) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionBlockMetrics_StateReads) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionBlockMetrics_StateReads) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.CodeBytes != nil { + size, err := (*wrapperspb.UInt64Value)(m.CodeBytes).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.Code != nil { + size, err := (*wrapperspb.UInt64Value)(m.Code).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.StorageSlots != nil { + size, err := (*wrapperspb.UInt64Value)(m.StorageSlots).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Accounts != nil { + size, err := (*wrapperspb.UInt64Value)(m.Accounts).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionBlockMetrics_StateWrites) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionBlockMetrics_StateWrites) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionBlockMetrics_StateWrites) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.CodeBytes != nil { + size, err := (*wrapperspb.UInt64Value)(m.CodeBytes).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.Code != nil { + size, err := (*wrapperspb.UInt64Value)(m.Code).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.StorageSlotsDeleted != nil { + size, err := (*wrapperspb.UInt64Value)(m.StorageSlotsDeleted).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.StorageSlots != nil { + size, err := (*wrapperspb.UInt64Value)(m.StorageSlots).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.AccountsDeleted != nil { + size, err := (*wrapperspb.UInt64Value)(m.AccountsDeleted).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Accounts != nil { + size, err := (*wrapperspb.UInt64Value)(m.Accounts).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionBlockMetrics_CacheEntry) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionBlockMetrics_CacheEntry) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionBlockMetrics_CacheEntry) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.HitRate != nil { + size, err := (*wrapperspb.DoubleValue)(m.HitRate).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Misses != nil { + size, err := (*wrapperspb.Int64Value)(m.Misses).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Hits != nil { + size, err := (*wrapperspb.Int64Value)(m.Hits).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionBlockMetrics_CodeCacheEntry) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionBlockMetrics_CodeCacheEntry) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionBlockMetrics_CodeCacheEntry) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.MissBytes != nil { + size, err := (*wrapperspb.Int64Value)(m.MissBytes).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.HitBytes != nil { + size, err := (*wrapperspb.Int64Value)(m.HitBytes).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.HitRate != nil { + size, err := (*wrapperspb.DoubleValue)(m.HitRate).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.Misses != nil { + size, err := (*wrapperspb.Int64Value)(m.Misses).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Hits != nil { + size, err := (*wrapperspb.Int64Value)(m.Hits).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExecutionBlockMetrics) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExecutionBlockMetrics) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ExecutionBlockMetrics) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.CodeCache != nil { + size, err := m.CodeCache.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if m.StorageCache != nil { + size, err := m.StorageCache.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + if m.AccountCache != nil { + size, err := m.AccountCache.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x72 + } + if m.StateWrites != nil { + size, err := m.StateWrites.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if m.StateReads != nil { + size, err := m.StateReads.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.MgasPerSec != nil { + size, err := (*wrapperspb.DoubleValue)(m.MgasPerSec).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + if m.TotalMs != nil { + size, err := (*wrapperspb.DoubleValue)(m.TotalMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } + if m.CommitMs != nil { + size, err := (*wrapperspb.DoubleValue)(m.CommitMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + if m.StateHashMs != nil { + size, err := (*wrapperspb.DoubleValue)(m.StateHashMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.StateReadMs != nil { + size, err := (*wrapperspb.DoubleValue)(m.StateReadMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.ExecutionMs != nil { + size, err := (*wrapperspb.DoubleValue)(m.ExecutionMs).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if m.TxCount != nil { + size, err := (*wrapperspb.UInt32Value)(m.TxCount).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.GasUsed != nil { + size, err := (*wrapperspb.UInt64Value)(m.GasUsed).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if len(m.BlockHash) > 0 { + i -= len(m.BlockHash) + copy(dAtA[i:], m.BlockHash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.BlockHash))) + i-- + dAtA[i] = 0x1a + } + if m.BlockNumber != nil { + size, err := (*wrapperspb.UInt64Value)(m.BlockNumber).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.Source) > 0 { + i -= len(m.Source) + copy(dAtA[i:], m.Source) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Source))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DecoratedEvent) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DecoratedEvent) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if vtmsg, ok := m.Data.(interface { + MarshalToSizedBufferVT([]byte) (int, error) + }); ok { + size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + } + if m.Meta != nil { + size, err := m.Meta.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Event != nil { + size, err := m.Event.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DecoratedEvent_EthV1EventsAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsAttestation != nil { + size, err := m.EthV1EventsAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsBlock != nil { + size, err := m.EthV1EventsBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsChainReorg) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsChainReorg) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsChainReorg != nil { + size, err := m.EthV1EventsChainReorg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsFinalizedCheckpoint) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsFinalizedCheckpoint) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsFinalizedCheckpoint != nil { + size, err := m.EthV1EventsFinalizedCheckpoint.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsHead) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsHead) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsHead != nil { + size, err := m.EthV1EventsHead.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsVoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsVoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsVoluntaryExit != nil { + size, err := m.EthV1EventsVoluntaryExit.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsContributionAndProof) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsContributionAndProof) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsContributionAndProof != nil { + size, err := m.EthV1EventsContributionAndProof.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_MempoolTransaction) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_MempoolTransaction) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.MempoolTransaction) + copy(dAtA[i:], m.MempoolTransaction) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MempoolTransaction))) + i-- + dAtA[i] = 0x52 + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlock != nil { + size, err := m.EthV2BeaconBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1ForkChoice) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1ForkChoice) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1ForkChoice != nil { + size, err := m.EthV1ForkChoice.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1ForkChoiceReorg) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1ForkChoiceReorg) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1ForkChoiceReorg != nil { + size, err := m.EthV1ForkChoiceReorg.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1BeaconCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1BeaconCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconCommittee != nil { + size, err := m.EthV1BeaconCommittee.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x72 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1ValidatorAttestationData) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1ValidatorAttestationData) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1ValidatorAttestationData != nil { + size, err := m.EthV1ValidatorAttestationData.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x7a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsAttestationV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsAttestationV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsAttestationV2 != nil { + size, err := m.EthV1EventsAttestationV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsBlockV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsBlockV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsBlockV2 != nil { + size, err := m.EthV1EventsBlockV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsChainReorgV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsChainReorgV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsChainReorgV2 != nil { + size, err := m.EthV1EventsChainReorgV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsFinalizedCheckpointV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsFinalizedCheckpointV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsFinalizedCheckpointV2 != nil { + size, err := m.EthV1EventsFinalizedCheckpointV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x9a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsHeadV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsHeadV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsHeadV2 != nil { + size, err := m.EthV1EventsHeadV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsVoluntaryExitV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsVoluntaryExitV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsVoluntaryExitV2 != nil { + size, err := m.EthV1EventsVoluntaryExitV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsContributionAndProofV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsContributionAndProofV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsContributionAndProofV2 != nil { + size, err := m.EthV1EventsContributionAndProofV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_MempoolTransactionV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_MempoolTransactionV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.MempoolTransactionV2) + copy(dAtA[i:], m.MempoolTransactionV2) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.MempoolTransactionV2))) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xba + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockV2 != nil { + size, err := m.EthV2BeaconBlockV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1ForkChoiceV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1ForkChoiceV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1ForkChoiceV2 != nil { + size, err := m.EthV1ForkChoiceV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1ForkChoiceReorgV2) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1ForkChoiceReorgV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1ForkChoiceReorgV2 != nil { + size, err := m.EthV1ForkChoiceReorgV2.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockAttesterSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockAttesterSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockAttesterSlashing != nil { + size, err := m.EthV2BeaconBlockAttesterSlashing.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockProposerSlashing) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockProposerSlashing) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockProposerSlashing != nil { + size, err := m.EthV2BeaconBlockProposerSlashing.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockVoluntaryExit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockVoluntaryExit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockVoluntaryExit != nil { + size, err := m.EthV2BeaconBlockVoluntaryExit.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockDeposit) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockDeposit) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockDeposit != nil { + size, err := m.EthV2BeaconBlockDeposit.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockBlsToExecutionChange) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockBlsToExecutionChange) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockBlsToExecutionChange != nil { + size, err := m.EthV2BeaconBlockBlsToExecutionChange.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xfa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockExecutionTransaction) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockExecutionTransaction) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockExecutionTransaction != nil { + size, err := m.EthV2BeaconBlockExecutionTransaction.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockWithdrawal) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockWithdrawal) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockWithdrawal != nil { + size, err := m.EthV2BeaconBlockWithdrawal.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsBlobSidecar != nil { + size, err := m.EthV1EventsBlobSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1BeaconBlockBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1BeaconBlockBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconBlockBlobSidecar != nil { + size, err := m.EthV1BeaconBlockBlobSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_BeaconP2PAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_BeaconP2PAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.BeaconP2PAttestation != nil { + size, err := m.BeaconP2PAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1ProposerDuty) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1ProposerDuty) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1ProposerDuty != nil { + size, err := m.EthV1ProposerDuty.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockElaboratedAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockElaboratedAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockElaboratedAttestation != nil { + size, err := m.EthV2BeaconBlockElaboratedAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xba + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceAddPeer) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceAddPeer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceAddPeer != nil { + size, err := m.Libp2PTraceAddPeer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRemovePeer) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRemovePeer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRemovePeer != nil { + size, err := m.Libp2PTraceRemovePeer.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRecvRpc) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRecvRpc) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRecvRpc != nil { + size, err := m.Libp2PTraceRecvRpc.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceSendRpc) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceSendRpc) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceSendRpc != nil { + size, err := m.Libp2PTraceSendRpc.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceJoin) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceJoin) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceJoin != nil { + size, err := m.Libp2PTraceJoin.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceConnected) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceConnected) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceConnected != nil { + size, err := m.Libp2PTraceConnected.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceDisconnected) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceDisconnected) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceDisconnected != nil { + size, err := m.Libp2PTraceDisconnected.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceHandleMetadata) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceHandleMetadata) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceHandleMetadata != nil { + size, err := m.Libp2PTraceHandleMetadata.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xfa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceHandleStatus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceHandleStatus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceHandleStatus != nil { + size, err := m.Libp2PTraceHandleStatus.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceGossipsubBeaconBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceGossipsubBeaconBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubBeaconBlock != nil { + size, err := m.Libp2PTraceGossipsubBeaconBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubBeaconAttestation != nil { + size, err := m.Libp2PTraceGossipsubBeaconAttestation.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceGossipsubBlobSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceGossipsubBlobSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubBlobSidecar != nil { + size, err := m.Libp2PTraceGossipsubBlobSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x9a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1Validators) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1Validators) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1Validators != nil { + size, err := m.EthV1Validators.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_MevRelayBidTraceBuilderBlockSubmission) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_MevRelayBidTraceBuilderBlockSubmission) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.MevRelayBidTraceBuilderBlockSubmission != nil { + size, err := m.MevRelayBidTraceBuilderBlockSubmission.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_MevRelayPayloadDelivered) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_MevRelayPayloadDelivered) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.MevRelayPayloadDelivered != nil { + size, err := m.MevRelayPayloadDelivered.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV3ValidatorBlock) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV3ValidatorBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV3ValidatorBlock != nil { + size, err := m.EthV3ValidatorBlock.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xba + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_MevRelayValidatorRegistration) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_MevRelayValidatorRegistration) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.MevRelayValidatorRegistration != nil { + size, err := m.MevRelayValidatorRegistration.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsBlockGossip) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsBlockGossip) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsBlockGossip != nil { + size, err := m.EthV1EventsBlockGossip.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceDropRpc) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceDropRpc) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceDropRpc != nil { + size, err := m.Libp2PTraceDropRpc.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceLeave) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceLeave) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceLeave != nil { + size, err := m.Libp2PTraceLeave.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceGraft) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceGraft) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGraft != nil { + size, err := m.Libp2PTraceGraft.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTracePrune) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTracePrune) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTracePrune != nil { + size, err := m.Libp2PTracePrune.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceDuplicateMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceDuplicateMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceDuplicateMessage != nil { + size, err := m.Libp2PTraceDuplicateMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceDeliverMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceDeliverMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceDeliverMessage != nil { + size, err := m.Libp2PTraceDeliverMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xfa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTracePublishMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTracePublishMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTracePublishMessage != nil { + size, err := m.Libp2PTracePublishMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRejectMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRejectMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRejectMessage != nil { + size, err := m.Libp2PTraceRejectMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIhave) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIhave) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlIhave != nil { + size, err := m.Libp2PTraceRpcMetaControlIhave.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x92 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIwant) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIwant) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlIwant != nil { + size, err := m.Libp2PTraceRpcMetaControlIwant.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x9a + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIdontwant) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIdontwant) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlIdontwant != nil { + size, err := m.Libp2PTraceRpcMetaControlIdontwant.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xa2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlGraft) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlGraft) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlGraft != nil { + size, err := m.Libp2PTraceRpcMetaControlGraft.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xaa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlPrune) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlPrune) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaControlPrune != nil { + size, err := m.Libp2PTraceRpcMetaControlPrune.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xb2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaSubscription) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRpcMetaSubscription) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaSubscription != nil { + size, err := m.Libp2PTraceRpcMetaSubscription.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xba + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaMessage) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRpcMetaMessage) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcMetaMessage != nil { + size, err := m.Libp2PTraceRpcMetaMessage.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_NodeRecordConsensus) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_NodeRecordConsensus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.NodeRecordConsensus != nil { + size, err := m.NodeRecordConsensus.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_NodeRecordExecution) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_NodeRecordExecution) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.NodeRecordExecution != nil { + size, err := m.NodeRecordExecution.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceGossipsubAggregateAndProof) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceGossipsubAggregateAndProof) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubAggregateAndProof != nil { + size, err := m.Libp2PTraceGossipsubAggregateAndProof.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1EventsDataColumnSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1EventsDataColumnSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1EventsDataColumnSidecar != nil { + size, err := m.EthV1EventsDataColumnSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceGossipsubDataColumnSidecar) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceGossipsubDataColumnSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceGossipsubDataColumnSidecar != nil { + size, err := m.Libp2PTraceGossipsubDataColumnSidecar.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceSyntheticHeartbeat) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceSyntheticHeartbeat) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceSyntheticHeartbeat != nil { + size, err := m.Libp2PTraceSyntheticHeartbeat.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceIdentify) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceIdentify) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceIdentify != nil { + size, err := m.Libp2PTraceIdentify.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xfa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_Libp2PTraceRpcDataColumnCustodyProbe) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_Libp2PTraceRpcDataColumnCustodyProbe) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Libp2PTraceRpcDataColumnCustodyProbe != nil { + size, err := m.Libp2PTraceRpcDataColumnCustodyProbe.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xc + i-- + dAtA[i] = 0xc2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_ExecutionStateSize) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_ExecutionStateSize) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ExecutionStateSize != nil { + size, err := m.ExecutionStateSize.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xc + i-- + dAtA[i] = 0xca + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_ConsensusEngineApiNewPayload) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_ConsensusEngineApiNewPayload) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ConsensusEngineApiNewPayload != nil { + size, err := m.ConsensusEngineApiNewPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xc + i-- + dAtA[i] = 0xd2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_ConsensusEngineApiGetBlobs) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_ConsensusEngineApiGetBlobs) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ConsensusEngineApiGetBlobs != nil { + size, err := m.ConsensusEngineApiGetBlobs.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xc + i-- + dAtA[i] = 0xda + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_ExecutionEngineNewPayload) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_ExecutionEngineNewPayload) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ExecutionEngineNewPayload != nil { + size, err := m.ExecutionEngineNewPayload.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xc + i-- + dAtA[i] = 0xe2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_ExecutionEngineGetBlobs) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_ExecutionEngineGetBlobs) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ExecutionEngineGetBlobs != nil { + size, err := m.ExecutionEngineGetBlobs.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xc + i-- + dAtA[i] = 0xea + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1BeaconBlob) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1BeaconBlob) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconBlob != nil { + size, err := m.EthV1BeaconBlob.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xc + i-- + dAtA[i] = 0xf2 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV1BeaconSyncCommittee) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV1BeaconSyncCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV1BeaconSyncCommittee != nil { + size, err := m.EthV1BeaconSyncCommittee.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xc + i-- + dAtA[i] = 0xfa + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_EthV2BeaconBlockSyncAggregate) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_EthV2BeaconBlockSyncAggregate) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.EthV2BeaconBlockSyncAggregate != nil { + size, err := m.EthV2BeaconBlockSyncAggregate.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xd + i-- + dAtA[i] = 0x82 + } + return len(dAtA) - i, nil +} +func (m *DecoratedEvent_ExecutionBlockMetrics) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *DecoratedEvent_ExecutionBlockMetrics) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ExecutionBlockMetrics != nil { + size, err := m.ExecutionBlockMetrics.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xd + i-- + dAtA[i] = 0x8a + } + return len(dAtA) - i, nil +} +func (m *CreateEventsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Events) > 0 { + for _, e := range m.Events { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *CreateEventsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EventsIngested != nil { + l = (*wrapperspb.UInt64Value)(m.EventsIngested).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Epoch) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Number != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Number)) + } + if m.StartDateTime != nil { + l = (*timestamppb.Timestamp)(m.StartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *EpochV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Number != nil { + l = (*wrapperspb.UInt64Value)(m.Number).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StartDateTime != nil { + l = (*timestamppb.Timestamp)(m.StartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Slot) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Number != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Number)) + } + if m.StartDateTime != nil { + l = (*timestamppb.Timestamp)(m.StartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SlotV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Number != nil { + l = (*wrapperspb.UInt64Value)(m.Number).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StartDateTime != nil { + l = (*timestamppb.Timestamp)(m.StartDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ForkID) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Hash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Next) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Propagation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SlotStartDiff != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.SlotStartDiff)) + } + n += len(m.unknownFields) + return n +} + +func (m *PropagationV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SlotStartDiff != nil { + l = (*wrapperspb.UInt64Value)(m.SlotStartDiff).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *AttestingValidator) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CommitteeIndex != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.CommitteeIndex)) + } + if m.Index != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Index)) + } + n += len(m.unknownFields) + return n +} + +func (m *AttestingValidatorV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CommitteeIndex != nil { + l = (*wrapperspb.UInt64Value)(m.CommitteeIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Index != nil { + l = (*wrapperspb.UInt64Value)(m.Index).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DebugForkChoiceReorg) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Before != nil { + l = m.Before.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.After != nil { + l = m.After.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Event != nil { + l = m.Event.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DebugForkChoiceReorgV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Before != nil { + l = m.Before.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.After != nil { + l = m.After.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Event != nil { + l = m.Event.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Validators) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Validators) > 0 { + for _, e := range m.Validators { + l = e.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *SyncCommitteeData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SyncCommittee != nil { + l = m.SyncCommittee.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SyncAggregateData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.SyncCommitteeBits) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.SyncCommitteeSignature) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ValidatorsParticipated) > 0 { + for _, e := range m.ValidatorsParticipated { + l = (*wrapperspb.UInt64Value)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if len(m.ValidatorsMissed) > 0 { + for _, e := range m.ValidatorsMissed { + l = (*wrapperspb.UInt64Value)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.ParticipationCount != nil { + l = (*wrapperspb.UInt64Value)(m.ParticipationCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *BlockIdentifier) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Root) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionStateSize) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.AccountBytes) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.AccountTrienodeBytes) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.AccountTrienodes) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Accounts) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockNumber) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ContractCodeBytes) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ContractCodes) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StorageBytes) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StorageTrienodeBytes) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StorageTrienodes) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Storages) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ConsensusEngineAPINewPayload) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RequestedAt != nil { + l = (*timestamppb.Timestamp)(m.RequestedAt).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.DurationMs != nil { + l = (*wrapperspb.UInt64Value)(m.DurationMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentBlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ProposerIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ProposerIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TxCount != nil { + l = (*wrapperspb.UInt32Value)(m.TxCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlobCount != nil { + l = (*wrapperspb.UInt32Value)(m.BlobCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Status) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LatestValidHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ValidationError) + if l > 0 { + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.MethodVersion) + if l > 0 { + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ConsensusEngineAPIGetBlobs) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RequestedAt != nil { + l = (*timestamppb.Timestamp)(m.RequestedAt).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.DurationMs != nil { + l = (*wrapperspb.UInt64Value)(m.DurationMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = (*wrapperspb.UInt64Value)(m.Slot).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentBlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestedCount != nil { + l = (*wrapperspb.UInt32Value)(m.RequestedCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.VersionedHashes) > 0 { + for _, s := range m.VersionedHashes { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.ReturnedCount != nil { + l = (*wrapperspb.UInt32Value)(m.ReturnedCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Status) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ErrorMessage) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.MethodVersion) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionEngineNewPayload) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Source != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Source)) + } + if m.RequestedAt != nil { + l = (*timestamppb.Timestamp)(m.RequestedAt).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.DurationMs != nil { + l = (*wrapperspb.UInt64Value)(m.DurationMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasLimit != nil { + l = (*wrapperspb.UInt64Value)(m.GasLimit).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TxCount != nil { + l = (*wrapperspb.UInt32Value)(m.TxCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlobCount != nil { + l = (*wrapperspb.UInt32Value)(m.BlobCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Status) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.LatestValidHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ValidationError) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.MethodVersion) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionEngineGetBlobs) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Source != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Source)) + } + if m.RequestedAt != nil { + l = (*timestamppb.Timestamp)(m.RequestedAt).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.DurationMs != nil { + l = (*wrapperspb.UInt64Value)(m.DurationMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestedCount != nil { + l = (*wrapperspb.UInt32Value)(m.RequestedCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.VersionedHashes) > 0 { + for _, s := range m.VersionedHashes { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + if m.ReturnedCount != nil { + l = (*wrapperspb.UInt32Value)(m.ReturnedCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Status) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ErrorMessage) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.MethodVersion) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.ReturnedBlobIndexes) > 0 { + for _, e := range m.ReturnedBlobIndexes { + l = (*wrapperspb.UInt32Value)(e).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_Ethereum_Network) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Id != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Id)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_Ethereum_Execution) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ForkId != nil { + l = m.ForkId.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Implementation) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.VersionMajor) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.VersionMinor) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.VersionPatch) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_Ethereum_Consensus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Implementation) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_Ethereum) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Network != nil { + l = m.Network.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Execution != nil { + l = m.Execution.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Consensus != nil { + l = m.Consensus.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AttestingValidator != nil { + l = m.AttestingValidator.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AttestingValidator != nil { + l = m.AttestingValidator.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Contribution != nil { + l = m.Contribution.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Contribution != nil { + l = m.Contribution.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_ForkChoiceSnapshot) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RequestEpoch != nil { + l = m.RequestEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestSlot != nil { + l = m.RequestSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestedAtSlotStartDiffMs != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.RequestedAtSlotStartDiffMs)) + } + if m.RequestDurationMs != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.RequestDurationMs)) + } + if m.Timestamp != nil { + l = (*timestamppb.Timestamp)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_ForkChoiceSnapshotV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RequestEpoch != nil { + l = m.RequestEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestSlot != nil { + l = m.RequestSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestedAtSlotStartDiffMs != nil { + l = (*wrapperspb.UInt64Value)(m.RequestedAtSlotStartDiffMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestDurationMs != nil { + l = (*wrapperspb.UInt64Value)(m.RequestDurationMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*timestamppb.Timestamp)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Snapshot != nil { + l = m.Snapshot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Snapshot != nil { + l = m.Snapshot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Before != nil { + l = m.Before.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.After != nil { + l = m.After.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Before != nil { + l = m.Before.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.After != nil { + l = m.After.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SyncCommitteePeriod != nil { + l = (*wrapperspb.UInt64Value)(m.SyncCommitteePeriod).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.SyncCommitteePeriod != nil { + l = (*wrapperspb.UInt64Value)(m.SyncCommitteePeriod).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalMempoolTransactionData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Hash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.From) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.To) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Nonce != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Nonce)) + } + l = len(m.GasPrice) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Gas != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Gas)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Size) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.CallDataSize) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalMempoolTransactionV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Hash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.From) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.To) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Nonce != nil { + l = (*wrapperspb.UInt64Value)(m.Nonce).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.GasPrice) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Gas != nil { + l = (*wrapperspb.UInt64Value)(m.Gas).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Size) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.CallDataSize) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Type != nil { + l = (*wrapperspb.UInt32Value)(m.Type).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.GasTipCap) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.GasFeeCap) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlobGas != nil { + l = (*wrapperspb.UInt64Value)(m.BlobGas).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlobGasFeeCap) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.BlobHashes) > 0 { + for _, s := range m.BlobHashes { + l = len(s) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + } + l = len(m.BlobSidecarsSize) + if l > 0 { + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlobSidecarsEmptySize) + if l > 0 { + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TransactionsCount != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.TransactionsCount)) + } + if m.TransactionsTotalBytes != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.TransactionsTotalBytes)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockRoot) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TransactionsCount != nil { + l = (*wrapperspb.UInt64Value)(m.TransactionsCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TransactionsTotalBytes != nil { + l = (*wrapperspb.UInt64Value)(m.TransactionsTotalBytes).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TransactionsTotalBytesCompressed != nil { + l = (*wrapperspb.UInt64Value)(m.TransactionsTotalBytesCompressed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TotalBytes != nil { + l = (*wrapperspb.UInt64Value)(m.TotalBytes).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TotalBytesCompressed != nil { + l = (*wrapperspb.UInt64Value)(m.TotalBytesCompressed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.FinalizedWhenRequested { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockDepositData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PositionInBlock != nil { + l = (*wrapperspb.UInt64Value)(m.PositionInBlock).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Size) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.CallDataSize) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlobSidecarsSize) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlobSidecarsEmptySize) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AttestationDataSnapshot) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RequestedAtSlotStartDiffMs != nil { + l = (*wrapperspb.UInt64Value)(m.RequestedAtSlotStartDiffMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestDurationMs != nil { + l = (*wrapperspb.UInt64Value)(m.RequestDurationMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Timestamp != nil { + l = (*timestamppb.Timestamp)(m.Timestamp).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Snapshot != nil { + l = m.Snapshot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.DataSize != nil { + l = (*wrapperspb.UInt64Value)(m.DataSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.VersionedHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.DataEmptySize != nil { + l = (*wrapperspb.UInt64Value)(m.DataEmptySize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalBeaconP2PAttestationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AttestingValidator != nil { + l = m.AttestingValidator.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Peer != nil { + l = m.Peer.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Subnet != nil { + l = (*wrapperspb.UInt32Value)(m.Subnet).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Validated != nil { + l = (*wrapperspb.BoolValue)(m.Validated).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1ProposerDutyData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.StateId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.PositionInBlock != nil { + l = (*wrapperspb.UInt64Value)(m.PositionInBlock).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceAddPeerData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRemovePeerData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRecvRPCData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceSendRPCData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceDropRPCData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceJoinData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceLeaveData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceGraftData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTracePruneData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceDeliverMessageData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTracePublishMessageData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRejectMessageData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceConnectedData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceDisconnectedData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleStatusData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceIdentifyData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageSize != nil { + l = (*wrapperspb.UInt32Value)(m.MessageSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Source != nil { + l = m.Source.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AttestingValidator != nil { + l = m.AttestingValidator.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageSize != nil { + l = (*wrapperspb.UInt32Value)(m.MessageSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AggregatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.AggregatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageSize != nil { + l = (*wrapperspb.UInt32Value)(m.MessageSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageSize != nil { + l = (*wrapperspb.UInt32Value)(m.MessageSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Propagation != nil { + l = m.Propagation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Metadata != nil { + l = m.Metadata.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Topic != nil { + l = (*wrapperspb.StringValue)(m.Topic).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageSize != nil { + l = (*wrapperspb.UInt32Value)(m.MessageSize).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MessageId != nil { + l = (*wrapperspb.StringValue)(m.MessageId).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1ValidatorsData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Relay != nil { + l = m.Relay.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestedAtSlotTime != nil { + l = (*wrapperspb.UInt64Value)(m.RequestedAtSlotTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ResponseAtSlotTime != nil { + l = (*wrapperspb.UInt64Value)(m.ResponseAtSlotTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Relay != nil { + l = m.Relay.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestedAtSlotTime != nil { + l = (*wrapperspb.UInt64Value)(m.RequestedAtSlotTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ResponseAtSlotTime != nil { + l = (*wrapperspb.UInt64Value)(m.ResponseAtSlotTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TransactionsCount != nil { + l = (*wrapperspb.UInt64Value)(m.TransactionsCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TransactionsTotalBytes != nil { + l = (*wrapperspb.UInt64Value)(m.TransactionsTotalBytes).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TransactionsTotalBytesCompressed != nil { + l = (*wrapperspb.UInt64Value)(m.TransactionsTotalBytesCompressed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TotalBytes != nil { + l = (*wrapperspb.UInt64Value)(m.TotalBytes).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TotalBytesCompressed != nil { + l = (*wrapperspb.UInt64Value)(m.TotalBytesCompressed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ExecutionValue) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ConsensusValue) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestDurationMs != nil { + l = (*wrapperspb.UInt64Value)(m.RequestDurationMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.RequestedAt != nil { + l = (*timestamppb.Timestamp)(m.RequestedAt).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Relay != nil { + l = m.Relay.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockSlot != nil { + l = m.WallclockSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.WallclockEpoch != nil { + l = m.WallclockEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ValidatorIndex != nil { + l = (*wrapperspb.UInt64Value)(m.ValidatorIndex).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalNodeRecordConsensusData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.FinalizedEpoch != nil { + l = m.FinalizedEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HeadSlot != nil { + l = m.HeadSlot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HeadEpoch != nil { + l = m.HeadEpoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Epoch != nil { + l = m.Epoch.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Slot != nil { + l = m.Slot.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Id) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Implementation) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Os) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ClockDrift != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ClockDrift)) + } + if m.Ethereum != nil { + l = m.Ethereum.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if len(m.Labels) > 0 { + for k, v := range m.Labels { + _ = k + _ = v + mapEntrySize := 1 + len(k) + protohelpers.SizeOfVarint(uint64(len(k))) + 1 + len(v) + protohelpers.SizeOfVarint(uint64(len(v))) + n += mapEntrySize + 1 + protohelpers.SizeOfVarint(uint64(mapEntrySize)) + } + } + if vtmsg, ok := m.AdditionalData.(interface{ SizeVT() int }); ok { + n += vtmsg.SizeVT() + } + if m.ModuleName != 0 { + n += 2 + protohelpers.SizeOfVarint(uint64(m.ModuleName)) + } + l = len(m.PresetName) + if l > 0 { + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ClientMeta_EthV1EventsAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsAttestation != nil { + l = m.EthV1EventsAttestation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsHead) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsHead != nil { + l = m.EthV1EventsHead.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsBlock != nil { + l = m.EthV1EventsBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsVoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsVoluntaryExit != nil { + l = m.EthV1EventsVoluntaryExit.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsFinalizedCheckpoint) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsFinalizedCheckpoint != nil { + l = m.EthV1EventsFinalizedCheckpoint.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsChainReorg) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsChainReorg != nil { + l = m.EthV1EventsChainReorg.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsContributionAndProof) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsContributionAndProof != nil { + l = m.EthV1EventsContributionAndProof.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_MempoolTransaction) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MempoolTransaction != nil { + l = m.MempoolTransaction.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlock != nil { + l = m.EthV2BeaconBlock.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1DebugForkChoice) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1DebugForkChoice != nil { + l = m.EthV1DebugForkChoice.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1DebugForkChoiceReorg) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1DebugForkChoiceReorg != nil { + l = m.EthV1DebugForkChoiceReorg.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1BeaconCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconCommittee != nil { + l = m.EthV1BeaconCommittee.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1ValidatorAttestationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1ValidatorAttestationData != nil { + l = m.EthV1ValidatorAttestationData.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsAttestationV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsAttestationV2 != nil { + l = m.EthV1EventsAttestationV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsHeadV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsHeadV2 != nil { + l = m.EthV1EventsHeadV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsBlockV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsBlockV2 != nil { + l = m.EthV1EventsBlockV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsVoluntaryExitV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsVoluntaryExitV2 != nil { + l = m.EthV1EventsVoluntaryExitV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsFinalizedCheckpointV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsFinalizedCheckpointV2 != nil { + l = m.EthV1EventsFinalizedCheckpointV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsChainReorgV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsChainReorgV2 != nil { + l = m.EthV1EventsChainReorgV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsContributionAndProofV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsContributionAndProofV2 != nil { + l = m.EthV1EventsContributionAndProofV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_MempoolTransactionV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MempoolTransactionV2 != nil { + l = m.MempoolTransactionV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockV2 != nil { + l = m.EthV2BeaconBlockV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1DebugForkChoiceV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1DebugForkChoiceV2 != nil { + l = m.EthV1DebugForkChoiceV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1DebugForkChoiceReorgV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1DebugForkChoiceReorgV2 != nil { + l = m.EthV1DebugForkChoiceReorgV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockAttesterSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockAttesterSlashing != nil { + l = m.EthV2BeaconBlockAttesterSlashing.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockProposerSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockProposerSlashing != nil { + l = m.EthV2BeaconBlockProposerSlashing.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockVoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockVoluntaryExit != nil { + l = m.EthV2BeaconBlockVoluntaryExit.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockDeposit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockDeposit != nil { + l = m.EthV2BeaconBlockDeposit.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockBlsToExecutionChange) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockBlsToExecutionChange != nil { + l = m.EthV2BeaconBlockBlsToExecutionChange.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockExecutionTransaction) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockExecutionTransaction != nil { + l = m.EthV2BeaconBlockExecutionTransaction.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockWithdrawal) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockWithdrawal != nil { + l = m.EthV2BeaconBlockWithdrawal.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsBlobSidecar != nil { + l = m.EthV1EventsBlobSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1BeaconBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconBlobSidecar != nil { + l = m.EthV1BeaconBlobSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_BeaconP2PAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BeaconP2PAttestation != nil { + l = m.BeaconP2PAttestation.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1ProposerDuty) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1ProposerDuty != nil { + l = m.EthV1ProposerDuty.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockElaboratedAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockElaboratedAttestation != nil { + l = m.EthV2BeaconBlockElaboratedAttestation.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceAddPeer) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceAddPeer != nil { + l = m.Libp2PTraceAddPeer.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRemovePeer) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRemovePeer != nil { + l = m.Libp2PTraceRemovePeer.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRecvRpc) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRecvRpc != nil { + l = m.Libp2PTraceRecvRpc.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceSendRpc) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceSendRpc != nil { + l = m.Libp2PTraceSendRpc.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceJoin) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceJoin != nil { + l = m.Libp2PTraceJoin.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceConnected) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceConnected != nil { + l = m.Libp2PTraceConnected.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceDisconnected) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceDisconnected != nil { + l = m.Libp2PTraceDisconnected.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceHandleMetadata) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceHandleMetadata != nil { + l = m.Libp2PTraceHandleMetadata.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceHandleStatus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceHandleStatus != nil { + l = m.Libp2PTraceHandleStatus.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceGossipsubBeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubBeaconBlock != nil { + l = m.Libp2PTraceGossipsubBeaconBlock.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceGossipsubBeaconAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubBeaconAttestation != nil { + l = m.Libp2PTraceGossipsubBeaconAttestation.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceGossipsubBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubBlobSidecar != nil { + l = m.Libp2PTraceGossipsubBlobSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1Validators) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1Validators != nil { + l = m.EthV1Validators.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_MevRelayBidTraceBuilderBlockSubmission) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MevRelayBidTraceBuilderBlockSubmission != nil { + l = m.MevRelayBidTraceBuilderBlockSubmission.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_MevRelayPayloadDelivered) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MevRelayPayloadDelivered != nil { + l = m.MevRelayPayloadDelivered.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV3ValidatorBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV3ValidatorBlock != nil { + l = m.EthV3ValidatorBlock.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_MevRelayValidatorRegistration) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MevRelayValidatorRegistration != nil { + l = m.MevRelayValidatorRegistration.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsBlockGossip) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsBlockGossip != nil { + l = m.EthV1EventsBlockGossip.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceDropRpc) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceDropRpc != nil { + l = m.Libp2PTraceDropRpc.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceLeave) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceLeave != nil { + l = m.Libp2PTraceLeave.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceGraft) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGraft != nil { + l = m.Libp2PTraceGraft.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTracePrune) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTracePrune != nil { + l = m.Libp2PTracePrune.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceDuplicateMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceDuplicateMessage != nil { + l = m.Libp2PTraceDuplicateMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceDeliverMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceDeliverMessage != nil { + l = m.Libp2PTraceDeliverMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTracePublishMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTracePublishMessage != nil { + l = m.Libp2PTracePublishMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRejectMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRejectMessage != nil { + l = m.Libp2PTraceRejectMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlIhave) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlIhave != nil { + l = m.Libp2PTraceRpcMetaControlIhave.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlIwant) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlIwant != nil { + l = m.Libp2PTraceRpcMetaControlIwant.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlIdontwant) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlIdontwant != nil { + l = m.Libp2PTraceRpcMetaControlIdontwant.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlGraft) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlGraft != nil { + l = m.Libp2PTraceRpcMetaControlGraft.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRpcMetaControlPrune) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlPrune != nil { + l = m.Libp2PTraceRpcMetaControlPrune.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRpcMetaSubscription) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaSubscription != nil { + l = m.Libp2PTraceRpcMetaSubscription.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRpcMetaMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaMessage != nil { + l = m.Libp2PTraceRpcMetaMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_NodeRecordConsensus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NodeRecordConsensus != nil { + l = m.NodeRecordConsensus.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceGossipsubAggregateAndProof) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubAggregateAndProof != nil { + l = m.Libp2PTraceGossipsubAggregateAndProof.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1EventsDataColumnSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsDataColumnSidecar != nil { + l = m.EthV1EventsDataColumnSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceGossipsubDataColumnSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubDataColumnSidecar != nil { + l = m.Libp2PTraceGossipsubDataColumnSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceSyntheticHeartbeat) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceSyntheticHeartbeat != nil { + l = m.Libp2PTraceSyntheticHeartbeat.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceRpcDataColumnCustodyProbe) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcDataColumnCustodyProbe != nil { + l = m.Libp2PTraceRpcDataColumnCustodyProbe.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_ConsensusEngineApiNewPayload) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ConsensusEngineApiNewPayload != nil { + l = m.ConsensusEngineApiNewPayload.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_ConsensusEngineApiGetBlobs) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ConsensusEngineApiGetBlobs != nil { + l = m.ConsensusEngineApiGetBlobs.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1BeaconBlob) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconBlob != nil { + l = m.EthV1BeaconBlob.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV1BeaconSyncCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconSyncCommittee != nil { + l = m.EthV1BeaconSyncCommittee.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_EthV2BeaconBlockSyncAggregate) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockSyncAggregate != nil { + l = m.EthV2BeaconBlockSyncAggregate.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ClientMeta_Libp2PTraceIdentify) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceIdentify != nil { + l = m.Libp2PTraceIdentify.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ServerMeta_Event) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ReceivedDateTime != nil { + l = (*timestamppb.Timestamp)(m.ReceivedDateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_Geo) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.City) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Country) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.CountryCode) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.ContinentCode) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Latitude != 0 { + n += 9 + } + if m.Longitude != 0 { + n += 9 + } + if m.AutonomousSystemNumber != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.AutonomousSystemNumber)) + } + l = len(m.AutonomousSystemOrganization) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_Client) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.IP) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Geo != nil { + l = m.Geo.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Group) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.User) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_Peer) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Geo != nil { + l = m.Geo.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_AdditionalBeaconP2PAttestationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Peer != nil { + l = m.Peer.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_AdditionalLibp2PTraceConnectedData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Peer != nil { + l = m.Peer.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_AdditionalLibp2PTraceDisconnectedData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Peer != nil { + l = m.Peer.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Peer != nil { + l = m.Peer.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_AdditionalLibp2PTraceIdentifyData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Peer != nil { + l = m.Peer.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_AdditionalNodeRecordConsensusData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Geo != nil { + l = m.Geo.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_AdditionalNodeRecordExecutionData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Geo != nil { + l = m.Geo.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Event != nil { + l = m.Event.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Client != nil { + l = m.Client.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if vtmsg, ok := m.AdditionalData.(interface{ SizeVT() int }); ok { + n += vtmsg.SizeVT() + } + n += len(m.unknownFields) + return n +} + +func (m *ServerMeta_BEACON_P2P_ATTESTATION) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BEACON_P2P_ATTESTATION != nil { + l = m.BEACON_P2P_ATTESTATION.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ServerMeta_LIBP2P_TRACE_CONNECTED) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LIBP2P_TRACE_CONNECTED != nil { + l = m.LIBP2P_TRACE_CONNECTED.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ServerMeta_LIBP2P_TRACE_DISCONNECTED) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LIBP2P_TRACE_DISCONNECTED != nil { + l = m.LIBP2P_TRACE_DISCONNECTED.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ServerMeta_NODE_RECORD_CONSENSUS) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NODE_RECORD_CONSENSUS != nil { + l = m.NODE_RECORD_CONSENSUS.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ServerMeta_NODE_RECORD_EXECUTION) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NODE_RECORD_EXECUTION != nil { + l = m.NODE_RECORD_EXECUTION.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ServerMeta_LIBP2P_TRACE_SYNTHETIC_HEARTBEAT) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LIBP2P_TRACE_SYNTHETIC_HEARTBEAT != nil { + l = m.LIBP2P_TRACE_SYNTHETIC_HEARTBEAT.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *ServerMeta_LIBP2P_TRACE_IDENTIFY) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LIBP2P_TRACE_IDENTIFY != nil { + l = m.LIBP2P_TRACE_IDENTIFY.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *Meta) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Client != nil { + l = m.Client.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Server != nil { + l = m.Server.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Event) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Name != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.Name)) + } + if m.DateTime != nil { + l = (*timestamppb.Timestamp)(m.DateTime).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Id) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionBlockMetrics_StateReads) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Accounts != nil { + l = (*wrapperspb.UInt64Value)(m.Accounts).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StorageSlots != nil { + l = (*wrapperspb.UInt64Value)(m.StorageSlots).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Code != nil { + l = (*wrapperspb.UInt64Value)(m.Code).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.CodeBytes != nil { + l = (*wrapperspb.UInt64Value)(m.CodeBytes).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionBlockMetrics_StateWrites) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Accounts != nil { + l = (*wrapperspb.UInt64Value)(m.Accounts).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AccountsDeleted != nil { + l = (*wrapperspb.UInt64Value)(m.AccountsDeleted).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StorageSlots != nil { + l = (*wrapperspb.UInt64Value)(m.StorageSlots).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StorageSlotsDeleted != nil { + l = (*wrapperspb.UInt64Value)(m.StorageSlotsDeleted).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Code != nil { + l = (*wrapperspb.UInt64Value)(m.Code).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.CodeBytes != nil { + l = (*wrapperspb.UInt64Value)(m.CodeBytes).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionBlockMetrics_CacheEntry) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Hits != nil { + l = (*wrapperspb.Int64Value)(m.Hits).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Misses != nil { + l = (*wrapperspb.Int64Value)(m.Misses).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HitRate != nil { + l = (*wrapperspb.DoubleValue)(m.HitRate).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionBlockMetrics_CodeCacheEntry) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Hits != nil { + l = (*wrapperspb.Int64Value)(m.Hits).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Misses != nil { + l = (*wrapperspb.Int64Value)(m.Misses).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HitRate != nil { + l = (*wrapperspb.DoubleValue)(m.HitRate).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HitBytes != nil { + l = (*wrapperspb.Int64Value)(m.HitBytes).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MissBytes != nil { + l = (*wrapperspb.Int64Value)(m.MissBytes).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecutionBlockMetrics) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Source) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.BlockNumber != nil { + l = (*wrapperspb.UInt64Value)(m.BlockNumber).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.BlockHash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.GasUsed != nil { + l = (*wrapperspb.UInt64Value)(m.GasUsed).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TxCount != nil { + l = (*wrapperspb.UInt32Value)(m.TxCount).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.ExecutionMs != nil { + l = (*wrapperspb.DoubleValue)(m.ExecutionMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StateReadMs != nil { + l = (*wrapperspb.DoubleValue)(m.StateReadMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StateHashMs != nil { + l = (*wrapperspb.DoubleValue)(m.StateHashMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.CommitMs != nil { + l = (*wrapperspb.DoubleValue)(m.CommitMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.TotalMs != nil { + l = (*wrapperspb.DoubleValue)(m.TotalMs).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.MgasPerSec != nil { + l = (*wrapperspb.DoubleValue)(m.MgasPerSec).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StateReads != nil { + l = m.StateReads.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StateWrites != nil { + l = m.StateWrites.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.AccountCache != nil { + l = m.AccountCache.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.StorageCache != nil { + l = m.StorageCache.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.CodeCache != nil { + l = m.CodeCache.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DecoratedEvent) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Event != nil { + l = m.Event.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.Meta != nil { + l = m.Meta.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if vtmsg, ok := m.Data.(interface{ SizeVT() int }); ok { + n += vtmsg.SizeVT() + } + n += len(m.unknownFields) + return n +} + +func (m *DecoratedEvent_EthV1EventsAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsAttestation != nil { + l = m.EthV1EventsAttestation.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsBlock != nil { + l = m.EthV1EventsBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsChainReorg) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsChainReorg != nil { + l = m.EthV1EventsChainReorg.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsFinalizedCheckpoint) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsFinalizedCheckpoint != nil { + l = m.EthV1EventsFinalizedCheckpoint.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsHead) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsHead != nil { + l = m.EthV1EventsHead.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsVoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsVoluntaryExit != nil { + l = m.EthV1EventsVoluntaryExit.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsContributionAndProof) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsContributionAndProof != nil { + l = m.EthV1EventsContributionAndProof.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_MempoolTransaction) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.MempoolTransaction) + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + return n +} +func (m *DecoratedEvent_EthV2BeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlock != nil { + l = m.EthV2BeaconBlock.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1ForkChoice) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1ForkChoice != nil { + l = m.EthV1ForkChoice.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1ForkChoiceReorg) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1ForkChoiceReorg != nil { + l = m.EthV1ForkChoiceReorg.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1BeaconCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconCommittee != nil { + l = m.EthV1BeaconCommittee.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1ValidatorAttestationData) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1ValidatorAttestationData != nil { + l = m.EthV1ValidatorAttestationData.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsAttestationV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsAttestationV2 != nil { + l = m.EthV1EventsAttestationV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsBlockV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsBlockV2 != nil { + l = m.EthV1EventsBlockV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsChainReorgV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsChainReorgV2 != nil { + l = m.EthV1EventsChainReorgV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsFinalizedCheckpointV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsFinalizedCheckpointV2 != nil { + l = m.EthV1EventsFinalizedCheckpointV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsHeadV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsHeadV2 != nil { + l = m.EthV1EventsHeadV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsVoluntaryExitV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsVoluntaryExitV2 != nil { + l = m.EthV1EventsVoluntaryExitV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsContributionAndProofV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsContributionAndProofV2 != nil { + l = m.EthV1EventsContributionAndProofV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_MempoolTransactionV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.MempoolTransactionV2) + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockV2 != nil { + l = m.EthV2BeaconBlockV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1ForkChoiceV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1ForkChoiceV2 != nil { + l = m.EthV1ForkChoiceV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1ForkChoiceReorgV2) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1ForkChoiceReorgV2 != nil { + l = m.EthV1ForkChoiceReorgV2.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockAttesterSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockAttesterSlashing != nil { + l = m.EthV2BeaconBlockAttesterSlashing.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockProposerSlashing) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockProposerSlashing != nil { + l = m.EthV2BeaconBlockProposerSlashing.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockVoluntaryExit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockVoluntaryExit != nil { + l = m.EthV2BeaconBlockVoluntaryExit.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockDeposit) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockDeposit != nil { + l = m.EthV2BeaconBlockDeposit.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockBlsToExecutionChange) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockBlsToExecutionChange != nil { + l = m.EthV2BeaconBlockBlsToExecutionChange.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockExecutionTransaction) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockExecutionTransaction != nil { + l = m.EthV2BeaconBlockExecutionTransaction.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockWithdrawal) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockWithdrawal != nil { + l = m.EthV2BeaconBlockWithdrawal.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsBlobSidecar != nil { + l = m.EthV1EventsBlobSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1BeaconBlockBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconBlockBlobSidecar != nil { + l = m.EthV1BeaconBlockBlobSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_BeaconP2PAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BeaconP2PAttestation != nil { + l = m.BeaconP2PAttestation.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1ProposerDuty) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1ProposerDuty != nil { + l = m.EthV1ProposerDuty.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockElaboratedAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockElaboratedAttestation != nil { + l = m.EthV2BeaconBlockElaboratedAttestation.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceAddPeer) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceAddPeer != nil { + l = m.Libp2PTraceAddPeer.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRemovePeer) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRemovePeer != nil { + l = m.Libp2PTraceRemovePeer.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRecvRpc) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRecvRpc != nil { + l = m.Libp2PTraceRecvRpc.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceSendRpc) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceSendRpc != nil { + l = m.Libp2PTraceSendRpc.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceJoin) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceJoin != nil { + l = m.Libp2PTraceJoin.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceConnected) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceConnected != nil { + l = m.Libp2PTraceConnected.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceDisconnected) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceDisconnected != nil { + l = m.Libp2PTraceDisconnected.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceHandleMetadata) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceHandleMetadata != nil { + l = m.Libp2PTraceHandleMetadata.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceHandleStatus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceHandleStatus != nil { + l = m.Libp2PTraceHandleStatus.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceGossipsubBeaconBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubBeaconBlock != nil { + l = m.Libp2PTraceGossipsubBeaconBlock.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubBeaconAttestation != nil { + l = m.Libp2PTraceGossipsubBeaconAttestation.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceGossipsubBlobSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubBlobSidecar != nil { + l = m.Libp2PTraceGossipsubBlobSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1Validators) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1Validators != nil { + l = m.EthV1Validators.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_MevRelayBidTraceBuilderBlockSubmission) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MevRelayBidTraceBuilderBlockSubmission != nil { + l = m.MevRelayBidTraceBuilderBlockSubmission.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_MevRelayPayloadDelivered) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MevRelayPayloadDelivered != nil { + l = m.MevRelayPayloadDelivered.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV3ValidatorBlock) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV3ValidatorBlock != nil { + l = m.EthV3ValidatorBlock.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_MevRelayValidatorRegistration) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MevRelayValidatorRegistration != nil { + l = m.MevRelayValidatorRegistration.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsBlockGossip) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsBlockGossip != nil { + l = m.EthV1EventsBlockGossip.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceDropRpc) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceDropRpc != nil { + l = m.Libp2PTraceDropRpc.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceLeave) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceLeave != nil { + l = m.Libp2PTraceLeave.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceGraft) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGraft != nil { + l = m.Libp2PTraceGraft.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTracePrune) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTracePrune != nil { + l = m.Libp2PTracePrune.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceDuplicateMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceDuplicateMessage != nil { + l = m.Libp2PTraceDuplicateMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceDeliverMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceDeliverMessage != nil { + l = m.Libp2PTraceDeliverMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTracePublishMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTracePublishMessage != nil { + l = m.Libp2PTracePublishMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRejectMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRejectMessage != nil { + l = m.Libp2PTraceRejectMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIhave) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlIhave != nil { + l = m.Libp2PTraceRpcMetaControlIhave.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIwant) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlIwant != nil { + l = m.Libp2PTraceRpcMetaControlIwant.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlIdontwant) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlIdontwant != nil { + l = m.Libp2PTraceRpcMetaControlIdontwant.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlGraft) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlGraft != nil { + l = m.Libp2PTraceRpcMetaControlGraft.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaControlPrune) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaControlPrune != nil { + l = m.Libp2PTraceRpcMetaControlPrune.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaSubscription) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaSubscription != nil { + l = m.Libp2PTraceRpcMetaSubscription.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRpcMetaMessage) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcMetaMessage != nil { + l = m.Libp2PTraceRpcMetaMessage.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_NodeRecordConsensus) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NodeRecordConsensus != nil { + l = m.NodeRecordConsensus.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_NodeRecordExecution) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NodeRecordExecution != nil { + l = m.NodeRecordExecution.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceGossipsubAggregateAndProof) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubAggregateAndProof != nil { + l = m.Libp2PTraceGossipsubAggregateAndProof.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1EventsDataColumnSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1EventsDataColumnSidecar != nil { + l = m.EthV1EventsDataColumnSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceGossipsubDataColumnSidecar) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceGossipsubDataColumnSidecar != nil { + l = m.Libp2PTraceGossipsubDataColumnSidecar.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceSyntheticHeartbeat) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceSyntheticHeartbeat != nil { + l = m.Libp2PTraceSyntheticHeartbeat.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceIdentify) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceIdentify != nil { + l = m.Libp2PTraceIdentify.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_Libp2PTraceRpcDataColumnCustodyProbe) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Libp2PTraceRpcDataColumnCustodyProbe != nil { + l = m.Libp2PTraceRpcDataColumnCustodyProbe.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_ExecutionStateSize) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ExecutionStateSize != nil { + l = m.ExecutionStateSize.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_ConsensusEngineApiNewPayload) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ConsensusEngineApiNewPayload != nil { + l = m.ConsensusEngineApiNewPayload.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_ConsensusEngineApiGetBlobs) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ConsensusEngineApiGetBlobs != nil { + l = m.ConsensusEngineApiGetBlobs.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_ExecutionEngineNewPayload) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ExecutionEngineNewPayload != nil { + l = m.ExecutionEngineNewPayload.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_ExecutionEngineGetBlobs) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ExecutionEngineGetBlobs != nil { + l = m.ExecutionEngineGetBlobs.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1BeaconBlob) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconBlob != nil { + l = m.EthV1BeaconBlob.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV1BeaconSyncCommittee) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV1BeaconSyncCommittee != nil { + l = m.EthV1BeaconSyncCommittee.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_EthV2BeaconBlockSyncAggregate) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EthV2BeaconBlockSyncAggregate != nil { + l = m.EthV2BeaconBlockSyncAggregate.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *DecoratedEvent_ExecutionBlockMetrics) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ExecutionBlockMetrics != nil { + l = m.ExecutionBlockMetrics.SizeVT() + n += 2 + l + protohelpers.SizeOfVarint(uint64(l)) + } + return n +} +func (m *CreateEventsRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateEventsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateEventsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Events = append(m.Events, &DecoratedEvent{}) + if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateEventsResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateEventsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateEventsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EventsIngested", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EventsIngested == nil { + m.EventsIngested = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.EventsIngested).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Epoch) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Epoch: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Epoch: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Number", wireType) + } + m.Number = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Number |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StartDateTime == nil { + m.StartDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.StartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *EpochV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EpochV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EpochV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Number", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Number == nil { + m.Number = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Number).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StartDateTime == nil { + m.StartDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.StartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Slot) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Slot: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Slot: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Number", wireType) + } + m.Number = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Number |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StartDateTime == nil { + m.StartDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.StartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SlotV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SlotV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SlotV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Number", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Number == nil { + m.Number = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Number).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StartDateTime == nil { + m.StartDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.StartDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ForkID) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ForkID: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ForkID: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Next", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Next = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Propagation) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Propagation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Propagation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SlotStartDiff", wireType) + } + m.SlotStartDiff = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SlotStartDiff |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *PropagationV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: PropagationV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PropagationV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SlotStartDiff", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SlotStartDiff == nil { + m.SlotStartDiff = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SlotStartDiff).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AttestingValidator) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AttestingValidator: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AttestingValidator: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CommitteeIndex", wireType) + } + m.CommitteeIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CommitteeIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + m.Index = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Index |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AttestingValidatorV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AttestingValidatorV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AttestingValidatorV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommitteeIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CommitteeIndex == nil { + m.CommitteeIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.CommitteeIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Index == nil { + m.Index = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Index).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DebugForkChoiceReorg) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DebugForkChoiceReorg: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DebugForkChoiceReorg: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Before", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Before == nil { + m.Before = &v1.ForkChoice{} + } + if err := m.Before.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field After", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.After == nil { + m.After = &v1.ForkChoice{} + } + if err := m.After.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Event == nil { + m.Event = &v1.EventChainReorg{} + } + if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DebugForkChoiceReorgV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DebugForkChoiceReorgV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DebugForkChoiceReorgV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Before", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Before == nil { + m.Before = &v1.ForkChoiceV2{} + } + if err := m.Before.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field After", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.After == nil { + m.After = &v1.ForkChoiceV2{} + } + if err := m.After.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Event == nil { + m.Event = &v1.EventChainReorgV2{} + } + if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Validators) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Validators: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Validators: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Validators", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Validators = append(m.Validators, &v1.Validator{}) + if err := m.Validators[len(m.Validators)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyncCommitteeData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncCommitteeData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncCommitteeData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncCommittee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncCommittee == nil { + m.SyncCommittee = &v1.SyncCommittee{} + } + if err := m.SyncCommittee.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyncAggregateData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncAggregateData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncAggregateData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncCommitteeBits", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SyncCommitteeBits = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncCommitteeSignature", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SyncCommitteeSignature = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorsParticipated", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ValidatorsParticipated = append(m.ValidatorsParticipated, &wrapperspb1.UInt64Value{}) + if err := (*wrapperspb.UInt64Value)(m.ValidatorsParticipated[len(m.ValidatorsParticipated)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorsMissed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ValidatorsMissed = append(m.ValidatorsMissed, &wrapperspb1.UInt64Value{}) + if err := (*wrapperspb.UInt64Value)(m.ValidatorsMissed[len(m.ValidatorsMissed)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParticipationCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ParticipationCount == nil { + m.ParticipationCount = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ParticipationCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlockIdentifier) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlockIdentifier: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlockIdentifier: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Root", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Root = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionStateSize) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionStateSize: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionStateSize: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AccountBytes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AccountBytes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AccountTrienodeBytes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AccountTrienodeBytes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AccountTrienodes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AccountTrienodes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Accounts", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Accounts = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockNumber = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ContractCodeBytes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ContractCodeBytes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ContractCodes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ContractCodes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StorageBytes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StorageBytes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StorageTrienodeBytes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StorageTrienodeBytes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StorageTrienodes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StorageTrienodes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Storages", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Storages = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ConsensusEngineAPINewPayload) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ConsensusEngineAPINewPayload: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ConsensusEngineAPINewPayload: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAt", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAt == nil { + m.RequestedAt = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.RequestedAt).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DurationMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DurationMs == nil { + m.DurationMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.DurationMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentBlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentBlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProposerIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ProposerIndex == nil { + m.ProposerIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ProposerIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TxCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TxCount == nil { + m.TxCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.TxCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlobCount == nil { + m.BlobCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.BlobCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Status = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LatestValidHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LatestValidHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidationError", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ValidationError = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MethodVersion", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MethodVersion = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ConsensusEngineAPIGetBlobs) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ConsensusEngineAPIGetBlobs: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ConsensusEngineAPIGetBlobs: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAt", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAt == nil { + m.RequestedAt = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.RequestedAt).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DurationMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DurationMs == nil { + m.DurationMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.DurationMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Slot).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentBlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentBlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedCount == nil { + m.RequestedCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.RequestedCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionedHashes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VersionedHashes = append(m.VersionedHashes, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReturnedCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ReturnedCount == nil { + m.ReturnedCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ReturnedCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Status = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ErrorMessage", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ErrorMessage = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MethodVersion", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MethodVersion = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionEngineNewPayload) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionEngineNewPayload: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionEngineNewPayload: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + m.Source = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Source |= EngineSource(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAt", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAt == nil { + m.RequestedAt = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.RequestedAt).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DurationMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DurationMs == nil { + m.DurationMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.DurationMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasLimit == nil { + m.GasLimit = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasLimit).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TxCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TxCount == nil { + m.TxCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.TxCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlobCount == nil { + m.BlobCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.BlobCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Status = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LatestValidHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LatestValidHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidationError", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ValidationError = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MethodVersion", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MethodVersion = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionEngineGetBlobs) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionEngineGetBlobs: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionEngineGetBlobs: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + m.Source = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Source |= EngineSource(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAt", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAt == nil { + m.RequestedAt = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.RequestedAt).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DurationMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DurationMs == nil { + m.DurationMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.DurationMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedCount == nil { + m.RequestedCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.RequestedCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionedHashes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VersionedHashes = append(m.VersionedHashes, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReturnedCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ReturnedCount == nil { + m.ReturnedCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.ReturnedCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Status = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ErrorMessage", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ErrorMessage = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MethodVersion", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MethodVersion = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReturnedBlobIndexes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReturnedBlobIndexes = append(m.ReturnedBlobIndexes, &wrapperspb1.UInt32Value{}) + if err := (*wrapperspb.UInt32Value)(m.ReturnedBlobIndexes[len(m.ReturnedBlobIndexes)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_Ethereum_Network) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_Ethereum_Network: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_Ethereum_Network: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + m.Id = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Id |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_Ethereum_Execution) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_Ethereum_Execution: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_Ethereum_Execution: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForkId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ForkId == nil { + m.ForkId = &ForkID{} + } + if err := m.ForkId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Implementation", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Implementation = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionMajor", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VersionMajor = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionMinor", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VersionMinor = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionPatch", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VersionPatch = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_Ethereum_Consensus) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_Ethereum_Consensus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_Ethereum_Consensus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Implementation", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Implementation = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_Ethereum) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_Ethereum: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_Ethereum: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Network", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Network == nil { + m.Network = &ClientMeta_Ethereum_Network{} + } + if err := m.Network.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Execution", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Execution == nil { + m.Execution = &ClientMeta_Ethereum_Execution{} + } + if err := m.Execution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Consensus", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Consensus == nil { + m.Consensus = &ClientMeta_Ethereum_Consensus{} + } + if err := m.Consensus.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1AttestationSourceData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1AttestationSourceData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1AttestationSourceData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1AttestationSourceV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1AttestationSourceV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1AttestationSourceV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1AttestationTargetData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1AttestationTargetData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1AttestationTargetData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1AttestationTargetV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1AttestationTargetV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1AttestationTargetV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsAttestationData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsAttestationData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsAttestationData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &ClientMeta_AdditionalEthV1AttestationSourceData{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &ClientMeta_AdditionalEthV1AttestationTargetData{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &Slot{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &Propagation{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttestingValidator", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AttestingValidator == nil { + m.AttestingValidator = &AttestingValidator{} + } + if err := m.AttestingValidator.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsAttestationV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsAttestationV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttestingValidator", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AttestingValidator == nil { + m.AttestingValidator = &AttestingValidatorV2{} + } + if err := m.AttestingValidator.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsHeadData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsHeadData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsHeadData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &Slot{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &Propagation{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsHeadV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsHeadV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsBlockData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsBlockData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsBlockData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &Slot{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &Propagation{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsBlockV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsBlockV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsBlockGossipData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsBlockGossipData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsVoluntaryExitData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsVoluntaryExitData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsChainReorgData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsChainReorgData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &Slot{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &Propagation{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsChainReorgV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsChainReorgV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &Slot{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &Propagation{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsContributionAndProofData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsContributionAndProofData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Contribution", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Contribution == nil { + m.Contribution = &ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData{} + } + if err := m.Contribution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Contribution", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Contribution == nil { + m.Contribution = &ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data{} + } + if err := m.Contribution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_ForkChoiceSnapshot) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_ForkChoiceSnapshot: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_ForkChoiceSnapshot: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestEpoch == nil { + m.RequestEpoch = &Epoch{} + } + if err := m.RequestEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestSlot == nil { + m.RequestSlot = &Slot{} + } + if err := m.RequestSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAtSlotStartDiffMs", wireType) + } + m.RequestedAtSlotStartDiffMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RequestedAtSlotStartDiffMs |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestDurationMs", wireType) + } + m.RequestDurationMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RequestDurationMs |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_ForkChoiceSnapshotV2) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_ForkChoiceSnapshotV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_ForkChoiceSnapshotV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestEpoch == nil { + m.RequestEpoch = &EpochV2{} + } + if err := m.RequestEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestSlot == nil { + m.RequestSlot = &SlotV2{} + } + if err := m.RequestSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAtSlotStartDiffMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAtSlotStartDiffMs == nil { + m.RequestedAtSlotStartDiffMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.RequestedAtSlotStartDiffMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestDurationMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestDurationMs == nil { + m.RequestDurationMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.RequestDurationMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1DebugForkChoiceData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1DebugForkChoiceData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Snapshot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Snapshot == nil { + m.Snapshot = &ClientMeta_ForkChoiceSnapshot{} + } + if err := m.Snapshot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1DebugForkChoiceV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1DebugForkChoiceV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Snapshot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Snapshot == nil { + m.Snapshot = &ClientMeta_ForkChoiceSnapshotV2{} + } + if err := m.Snapshot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Before", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Before == nil { + m.Before = &ClientMeta_ForkChoiceSnapshot{} + } + if err := m.Before.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field After", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.After == nil { + m.After = &ClientMeta_ForkChoiceSnapshot{} + } + if err := m.After.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Before", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Before == nil { + m.Before = &ClientMeta_ForkChoiceSnapshotV2{} + } + if err := m.Before.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field After", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.After == nil { + m.After = &ClientMeta_ForkChoiceSnapshotV2{} + } + if err := m.After.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1BeaconCommitteeData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1BeaconCommitteeData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1BeaconSyncCommitteeData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1BeaconSyncCommitteeData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncCommitteePeriod", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncCommitteePeriod == nil { + m.SyncCommitteePeriod = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SyncCommitteePeriod).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncCommitteePeriod", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SyncCommitteePeriod == nil { + m.SyncCommitteePeriod = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.SyncCommitteePeriod).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalMempoolTransactionData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalMempoolTransactionData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalMempoolTransactionData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field From", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.From = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field To", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.To = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Nonce", wireType) + } + m.Nonce = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Nonce |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasPrice", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.GasPrice = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Gas", wireType) + } + m.Gas = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Gas |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Size", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Size = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CallDataSize", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CallDataSize = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalMempoolTransactionV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalMempoolTransactionV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalMempoolTransactionV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field From", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.From = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field To", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.To = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Nonce", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Nonce == nil { + m.Nonce = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Nonce).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasPrice", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.GasPrice = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Gas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Gas == nil { + m.Gas = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Gas).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Size", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Size = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CallDataSize", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CallDataSize = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Type == nil { + m.Type = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Type).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasTipCap", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.GasTipCap = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasFeeCap", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.GasFeeCap = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobGas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlobGas == nil { + m.BlobGas = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlobGas).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobGasFeeCap", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobGasFeeCap = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobHashes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobHashes = append(m.BlobHashes, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobSidecarsSize", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobSidecarsSize = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobSidecarsEmptySize", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobSidecarsEmptySize = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &Epoch{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &Slot{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsCount", wireType) + } + m.TransactionsCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TransactionsCount |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsTotalBytes", wireType) + } + m.TransactionsTotalBytes = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TransactionsTotalBytes |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockV2Data: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockV2Data: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockRoot", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockRoot = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TransactionsCount == nil { + m.TransactionsCount = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TransactionsCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsTotalBytes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TransactionsTotalBytes == nil { + m.TransactionsTotalBytes = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TransactionsTotalBytes).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsTotalBytesCompressed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TransactionsTotalBytesCompressed == nil { + m.TransactionsTotalBytesCompressed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TransactionsTotalBytesCompressed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalBytes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TotalBytes == nil { + m.TotalBytes = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TotalBytes).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalBytesCompressed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TotalBytesCompressed == nil { + m.TotalBytesCompressed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TotalBytesCompressed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedWhenRequested", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.FinalizedWhenRequested = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockDepositData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockDepositData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockDepositData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PositionInBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PositionInBlock == nil { + m.PositionInBlock = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.PositionInBlock).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Size", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Size = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CallDataSize", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CallDataSize = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobSidecarsSize", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobSidecarsSize = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlobSidecarsEmptySize", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlobSidecarsEmptySize = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AttestationDataSnapshot) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AttestationDataSnapshot: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AttestationDataSnapshot: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAtSlotStartDiffMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAtSlotStartDiffMs == nil { + m.RequestedAtSlotStartDiffMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.RequestedAtSlotStartDiffMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestDurationMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestDurationMs == nil { + m.RequestDurationMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.RequestDurationMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timestamp == nil { + m.Timestamp = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.Timestamp).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1ValidatorAttestationDataData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1ValidatorAttestationDataData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Snapshot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Snapshot == nil { + m.Snapshot = &ClientMeta_AttestationDataSnapshot{} + } + if err := m.Snapshot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsBlobSidecarData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsBlobSidecarData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsDataColumnSidecarData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1EventsDataColumnSidecarData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1BeaconBlobSidecarData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1BeaconBlobSidecarData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DataSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DataSize == nil { + m.DataSize = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.DataSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionedHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VersionedHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DataEmptySize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DataEmptySize == nil { + m.DataEmptySize = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.DataEmptySize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalBeaconP2PAttestationData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalBeaconP2PAttestationData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttestingValidator", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AttestingValidator == nil { + m.AttestingValidator = &AttestingValidatorV2{} + } + if err := m.AttestingValidator.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Peer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Peer == nil { + m.Peer = &libp2p.Peer{} + } + if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Subnet", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Subnet == nil { + m.Subnet = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.Subnet).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Validated", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Validated == nil { + m.Validated = &wrapperspb1.BoolValue{} + } + if err := (*wrapperspb.BoolValue)(m.Validated).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1ProposerDutyData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1ProposerDutyData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1ProposerDutyData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &BlockIdentifier{} + } + if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PositionInBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PositionInBlock == nil { + m.PositionInBlock = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.PositionInBlock).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceAddPeerData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceAddPeerData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceAddPeerData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRemovePeerData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRemovePeerData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRemovePeerData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRecvRPCData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRecvRPCData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRecvRPCData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceSendRPCData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceSendRPCData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceSendRPCData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceDropRPCData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceDropRPCData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceDropRPCData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceJoinData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceJoinData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceJoinData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceLeaveData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceLeaveData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceLeaveData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceGraftData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGraftData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGraftData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTracePruneData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTracePruneData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTracePruneData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceDuplicateMessageData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceDuplicateMessageData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceDeliverMessageData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceDeliverMessageData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceDeliverMessageData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTracePublishMessageData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTracePublishMessageData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTracePublishMessageData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRejectMessageData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRejectMessageData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRejectMessageData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceConnectedData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceConnectedData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceConnectedData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceDisconnectedData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceDisconnectedData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceDisconnectedData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceHandleMetadataData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceHandleMetadataData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceHandleStatusData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceHandleStatusData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceHandleStatusData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceIdentifyData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceIdentifyData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceIdentifyData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageSize == nil { + m.MessageSize = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MessageSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Source == nil { + m.Source = &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttestingValidator", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AttestingValidator == nil { + m.AttestingValidator = &AttestingValidatorV2{} + } + if err := m.AttestingValidator.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageSize == nil { + m.MessageSize = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MessageSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AggregatorIndex == nil { + m.AggregatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.AggregatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageSize == nil { + m.MessageSize = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MessageSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageSize == nil { + m.MessageSize = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MessageSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Propagation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Propagation == nil { + m.Propagation = &PropagationV2{} + } + if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Metadata == nil { + m.Metadata = &libp2p.TraceEventMetadata{} + } + if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topic", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Topic == nil { + m.Topic = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.Topic).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageSize == nil { + m.MessageSize = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.MessageSize).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MessageId == nil { + m.MessageId = &wrapperspb1.StringValue{} + } + if err := (*wrapperspb.StringValue)(m.MessageId).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1ValidatorsData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1ValidatorsData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1ValidatorsData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Relay", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Relay == nil { + m.Relay = &mevrelay.Relay{} + } + if err := m.Relay.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAtSlotTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAtSlotTime == nil { + m.RequestedAtSlotTime = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.RequestedAtSlotTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResponseAtSlotTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ResponseAtSlotTime == nil { + m.ResponseAtSlotTime = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ResponseAtSlotTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalMevRelayPayloadDeliveredData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalMevRelayPayloadDeliveredData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Relay", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Relay == nil { + m.Relay = &mevrelay.Relay{} + } + if err := m.Relay.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAtSlotTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAtSlotTime == nil { + m.RequestedAtSlotTime = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.RequestedAtSlotTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResponseAtSlotTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ResponseAtSlotTime == nil { + m.ResponseAtSlotTime = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ResponseAtSlotTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV3ValidatorBlockData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV3ValidatorBlockData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TransactionsCount == nil { + m.TransactionsCount = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TransactionsCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsTotalBytes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TransactionsTotalBytes == nil { + m.TransactionsTotalBytes = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TransactionsTotalBytes).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionsTotalBytesCompressed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TransactionsTotalBytesCompressed == nil { + m.TransactionsTotalBytesCompressed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TransactionsTotalBytesCompressed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalBytes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TotalBytes == nil { + m.TotalBytes = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TotalBytes).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalBytesCompressed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TotalBytesCompressed == nil { + m.TotalBytesCompressed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.TotalBytesCompressed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionValue", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExecutionValue = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsensusValue", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConsensusValue = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestDurationMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestDurationMs == nil { + m.RequestDurationMs = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.RequestDurationMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestedAt", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RequestedAt == nil { + m.RequestedAt = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.RequestedAt).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalMevRelayValidatorRegistrationData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalMevRelayValidatorRegistrationData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Relay", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Relay == nil { + m.Relay = &mevrelay.Relay{} + } + if err := m.Relay.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockSlot == nil { + m.WallclockSlot = &SlotV2{} + } + if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WallclockEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WallclockEpoch == nil { + m.WallclockEpoch = &EpochV2{} + } + if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ValidatorIndex == nil { + m.ValidatorIndex = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.ValidatorIndex).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalNodeRecordConsensusData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalNodeRecordConsensusData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalNodeRecordConsensusData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FinalizedEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FinalizedEpoch == nil { + m.FinalizedEpoch = &EpochV2{} + } + if err := m.FinalizedEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadSlot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HeadSlot == nil { + m.HeadSlot = &SlotV2{} + } + if err := m.HeadSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HeadEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HeadEpoch == nil { + m.HeadEpoch = &EpochV2{} + } + if err := m.HeadEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalConsensusEngineAPINewPayloadData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalConsensusEngineAPINewPayloadData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalConsensusEngineAPIGetBlobsData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalConsensusEngineAPIGetBlobsData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta_AdditionalEthV1BeaconBlobData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1BeaconBlobData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta_AdditionalEthV1BeaconBlobData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Epoch == nil { + m.Epoch = &EpochV2{} + } + if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Slot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Slot == nil { + m.Slot = &SlotV2{} + } + if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Id = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Implementation", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Implementation = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Os", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Os = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ClockDrift", wireType) + } + m.ClockDrift = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ClockDrift |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Ethereum", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Ethereum == nil { + m.Ethereum = &ClientMeta_Ethereum{} + } + if err := m.Ethereum.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Labels == nil { + m.Labels = make(map[string]string) + } + var mapkey string + var mapvalue string + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return protohelpers.ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return protohelpers.ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var stringLenmapvalue uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapvalue |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapvalue := int(stringLenmapvalue) + if intStringLenmapvalue < 0 { + return protohelpers.ErrInvalidLength + } + postStringIndexmapvalue := iNdEx + intStringLenmapvalue + if postStringIndexmapvalue < 0 { + return protohelpers.ErrInvalidLength + } + if postStringIndexmapvalue > l { + return io.ErrUnexpectedEOF + } + mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) + iNdEx = postStringIndexmapvalue + } else { + iNdEx = entryPreIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.Labels[mapkey] = mapvalue + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsAttestation); ok { + if err := oneof.EthV1EventsAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsAttestationData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsAttestation{EthV1EventsAttestation: v} + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsHead", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsHead); ok { + if err := oneof.EthV1EventsHead.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsHeadData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsHead{EthV1EventsHead: v} + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsBlock); ok { + if err := oneof.EthV1EventsBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsBlockData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsBlock{EthV1EventsBlock: v} + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsVoluntaryExit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsVoluntaryExit); ok { + if err := oneof.EthV1EventsVoluntaryExit.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsVoluntaryExitData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsVoluntaryExit{EthV1EventsVoluntaryExit: v} + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsFinalizedCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsFinalizedCheckpoint); ok { + if err := oneof.EthV1EventsFinalizedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsFinalizedCheckpoint{EthV1EventsFinalizedCheckpoint: v} + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsChainReorg", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsChainReorg); ok { + if err := oneof.EthV1EventsChainReorg.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsChainReorgData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsChainReorg{EthV1EventsChainReorg: v} + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsContributionAndProof", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsContributionAndProof); ok { + if err := oneof.EthV1EventsContributionAndProof.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsContributionAndProofData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsContributionAndProof{EthV1EventsContributionAndProof: v} + } + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MempoolTransaction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MempoolTransaction); ok { + if err := oneof.MempoolTransaction.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalMempoolTransactionData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_MempoolTransaction{MempoolTransaction: v} + } + iNdEx = postIndex + case 18: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlock); ok { + if err := oneof.EthV2BeaconBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlock{EthV2BeaconBlock: v} + } + iNdEx = postIndex + case 19: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1DebugForkChoice", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1DebugForkChoice); ok { + if err := oneof.EthV1DebugForkChoice.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1DebugForkChoiceData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1DebugForkChoice{EthV1DebugForkChoice: v} + } + iNdEx = postIndex + case 20: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1DebugForkChoiceReorg", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1DebugForkChoiceReorg); ok { + if err := oneof.EthV1DebugForkChoiceReorg.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1DebugForkChoiceReorg{EthV1DebugForkChoiceReorg: v} + } + iNdEx = postIndex + case 21: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconCommittee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1BeaconCommittee); ok { + if err := oneof.EthV1BeaconCommittee.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1BeaconCommitteeData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1BeaconCommittee{EthV1BeaconCommittee: v} + } + iNdEx = postIndex + case 22: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1ValidatorAttestationData", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1ValidatorAttestationData); ok { + if err := oneof.EthV1ValidatorAttestationData.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1ValidatorAttestationDataData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1ValidatorAttestationData{EthV1ValidatorAttestationData: v} + } + iNdEx = postIndex + case 24: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsAttestationV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsAttestationV2); ok { + if err := oneof.EthV1EventsAttestationV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsAttestationV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsAttestationV2{EthV1EventsAttestationV2: v} + } + iNdEx = postIndex + case 25: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsHeadV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsHeadV2); ok { + if err := oneof.EthV1EventsHeadV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsHeadV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsHeadV2{EthV1EventsHeadV2: v} + } + iNdEx = postIndex + case 26: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsBlockV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsBlockV2); ok { + if err := oneof.EthV1EventsBlockV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsBlockV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsBlockV2{EthV1EventsBlockV2: v} + } + iNdEx = postIndex + case 27: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsVoluntaryExitV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsVoluntaryExitV2); ok { + if err := oneof.EthV1EventsVoluntaryExitV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsVoluntaryExitV2{EthV1EventsVoluntaryExitV2: v} + } + iNdEx = postIndex + case 28: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsFinalizedCheckpointV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsFinalizedCheckpointV2); ok { + if err := oneof.EthV1EventsFinalizedCheckpointV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsFinalizedCheckpointV2{EthV1EventsFinalizedCheckpointV2: v} + } + iNdEx = postIndex + case 29: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsChainReorgV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsChainReorgV2); ok { + if err := oneof.EthV1EventsChainReorgV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsChainReorgV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsChainReorgV2{EthV1EventsChainReorgV2: v} + } + iNdEx = postIndex + case 30: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsContributionAndProofV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsContributionAndProofV2); ok { + if err := oneof.EthV1EventsContributionAndProofV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsContributionAndProofV2{EthV1EventsContributionAndProofV2: v} + } + iNdEx = postIndex + case 31: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MempoolTransactionV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MempoolTransactionV2); ok { + if err := oneof.MempoolTransactionV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalMempoolTransactionV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_MempoolTransactionV2{MempoolTransactionV2: v} + } + iNdEx = postIndex + case 32: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockV2); ok { + if err := oneof.EthV2BeaconBlockV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockV2{EthV2BeaconBlockV2: v} + } + iNdEx = postIndex + case 33: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1DebugForkChoiceV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1DebugForkChoiceV2); ok { + if err := oneof.EthV1DebugForkChoiceV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1DebugForkChoiceV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1DebugForkChoiceV2{EthV1DebugForkChoiceV2: v} + } + iNdEx = postIndex + case 34: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1DebugForkChoiceReorgV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1DebugForkChoiceReorgV2); ok { + if err := oneof.EthV1DebugForkChoiceReorgV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1DebugForkChoiceReorgV2{EthV1DebugForkChoiceReorgV2: v} + } + iNdEx = postIndex + case 35: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockAttesterSlashing", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockAttesterSlashing); ok { + if err := oneof.EthV2BeaconBlockAttesterSlashing.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockAttesterSlashing{EthV2BeaconBlockAttesterSlashing: v} + } + iNdEx = postIndex + case 36: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockProposerSlashing", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockProposerSlashing); ok { + if err := oneof.EthV2BeaconBlockProposerSlashing.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockProposerSlashing{EthV2BeaconBlockProposerSlashing: v} + } + iNdEx = postIndex + case 37: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockVoluntaryExit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockVoluntaryExit); ok { + if err := oneof.EthV2BeaconBlockVoluntaryExit.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockVoluntaryExit{EthV2BeaconBlockVoluntaryExit: v} + } + iNdEx = postIndex + case 38: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockDeposit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockDeposit); ok { + if err := oneof.EthV2BeaconBlockDeposit.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockDepositData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockDeposit{EthV2BeaconBlockDeposit: v} + } + iNdEx = postIndex + case 39: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockBlsToExecutionChange", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockBlsToExecutionChange); ok { + if err := oneof.EthV2BeaconBlockBlsToExecutionChange.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockBlsToExecutionChange{EthV2BeaconBlockBlsToExecutionChange: v} + } + iNdEx = postIndex + case 40: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockExecutionTransaction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockExecutionTransaction); ok { + if err := oneof.EthV2BeaconBlockExecutionTransaction.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockExecutionTransaction{EthV2BeaconBlockExecutionTransaction: v} + } + iNdEx = postIndex + case 41: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockWithdrawal", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockWithdrawal); ok { + if err := oneof.EthV2BeaconBlockWithdrawal.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockWithdrawal{EthV2BeaconBlockWithdrawal: v} + } + iNdEx = postIndex + case 42: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsBlobSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsBlobSidecar); ok { + if err := oneof.EthV1EventsBlobSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsBlobSidecarData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsBlobSidecar{EthV1EventsBlobSidecar: v} + } + iNdEx = postIndex + case 44: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconBlobSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1BeaconBlobSidecar); ok { + if err := oneof.EthV1BeaconBlobSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1BeaconBlobSidecarData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1BeaconBlobSidecar{EthV1BeaconBlobSidecar: v} + } + iNdEx = postIndex + case 45: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeaconP2PAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_BeaconP2PAttestation); ok { + if err := oneof.BeaconP2PAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalBeaconP2PAttestationData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_BeaconP2PAttestation{BeaconP2PAttestation: v} + } + iNdEx = postIndex + case 46: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1ProposerDuty", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1ProposerDuty); ok { + if err := oneof.EthV1ProposerDuty.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1ProposerDutyData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1ProposerDuty{EthV1ProposerDuty: v} + } + iNdEx = postIndex + case 47: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockElaboratedAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockElaboratedAttestation); ok { + if err := oneof.EthV2BeaconBlockElaboratedAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockElaboratedAttestation{EthV2BeaconBlockElaboratedAttestation: v} + } + iNdEx = postIndex + case 48: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceAddPeer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceAddPeer); ok { + if err := oneof.Libp2PTraceAddPeer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceAddPeerData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceAddPeer{Libp2PTraceAddPeer: v} + } + iNdEx = postIndex + case 49: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRemovePeer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRemovePeer); ok { + if err := oneof.Libp2PTraceRemovePeer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRemovePeerData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRemovePeer{Libp2PTraceRemovePeer: v} + } + iNdEx = postIndex + case 50: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRecvRpc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRecvRpc); ok { + if err := oneof.Libp2PTraceRecvRpc.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRecvRPCData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRecvRpc{Libp2PTraceRecvRpc: v} + } + iNdEx = postIndex + case 51: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceSendRpc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceSendRpc); ok { + if err := oneof.Libp2PTraceSendRpc.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceSendRPCData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceSendRpc{Libp2PTraceSendRpc: v} + } + iNdEx = postIndex + case 52: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceJoin", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceJoin); ok { + if err := oneof.Libp2PTraceJoin.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceJoinData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceJoin{Libp2PTraceJoin: v} + } + iNdEx = postIndex + case 53: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceConnected", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceConnected); ok { + if err := oneof.Libp2PTraceConnected.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceConnectedData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceConnected{Libp2PTraceConnected: v} + } + iNdEx = postIndex + case 54: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceDisconnected", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceDisconnected); ok { + if err := oneof.Libp2PTraceDisconnected.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceDisconnectedData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceDisconnected{Libp2PTraceDisconnected: v} + } + iNdEx = postIndex + case 55: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceHandleMetadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceHandleMetadata); ok { + if err := oneof.Libp2PTraceHandleMetadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceHandleMetadataData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceHandleMetadata{Libp2PTraceHandleMetadata: v} + } + iNdEx = postIndex + case 56: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceHandleStatus", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceHandleStatus); ok { + if err := oneof.Libp2PTraceHandleStatus.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceHandleStatusData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceHandleStatus{Libp2PTraceHandleStatus: v} + } + iNdEx = postIndex + case 57: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubBeaconBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubBeaconBlock); ok { + if err := oneof.Libp2PTraceGossipsubBeaconBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceGossipsubBeaconBlock{Libp2PTraceGossipsubBeaconBlock: v} + } + iNdEx = postIndex + case 58: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubBeaconAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubBeaconAttestation); ok { + if err := oneof.Libp2PTraceGossipsubBeaconAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceGossipsubBeaconAttestation{Libp2PTraceGossipsubBeaconAttestation: v} + } + iNdEx = postIndex + case 59: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubBlobSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubBlobSidecar); ok { + if err := oneof.Libp2PTraceGossipsubBlobSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceGossipsubBlobSidecar{Libp2PTraceGossipsubBlobSidecar: v} + } + iNdEx = postIndex + case 60: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1Validators", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1Validators); ok { + if err := oneof.EthV1Validators.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1ValidatorsData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1Validators{EthV1Validators: v} + } + iNdEx = postIndex + case 61: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MevRelayBidTraceBuilderBlockSubmission", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MevRelayBidTraceBuilderBlockSubmission); ok { + if err := oneof.MevRelayBidTraceBuilderBlockSubmission.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_MevRelayBidTraceBuilderBlockSubmission{MevRelayBidTraceBuilderBlockSubmission: v} + } + iNdEx = postIndex + case 62: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MevRelayPayloadDelivered", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MevRelayPayloadDelivered); ok { + if err := oneof.MevRelayPayloadDelivered.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalMevRelayPayloadDeliveredData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_MevRelayPayloadDelivered{MevRelayPayloadDelivered: v} + } + iNdEx = postIndex + case 63: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ModuleName", wireType) + } + m.ModuleName = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ModuleName |= ModuleName(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 64: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PresetName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PresetName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 65: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV3ValidatorBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV3ValidatorBlock); ok { + if err := oneof.EthV3ValidatorBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV3ValidatorBlockData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV3ValidatorBlock{EthV3ValidatorBlock: v} + } + iNdEx = postIndex + case 66: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MevRelayValidatorRegistration", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MevRelayValidatorRegistration); ok { + if err := oneof.MevRelayValidatorRegistration.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalMevRelayValidatorRegistrationData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_MevRelayValidatorRegistration{MevRelayValidatorRegistration: v} + } + iNdEx = postIndex + case 67: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsBlockGossip", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsBlockGossip); ok { + if err := oneof.EthV1EventsBlockGossip.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsBlockGossipData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsBlockGossip{EthV1EventsBlockGossip: v} + } + iNdEx = postIndex + case 68: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceDropRpc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceDropRpc); ok { + if err := oneof.Libp2PTraceDropRpc.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceDropRPCData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceDropRpc{Libp2PTraceDropRpc: v} + } + iNdEx = postIndex + case 69: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceLeave", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceLeave); ok { + if err := oneof.Libp2PTraceLeave.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceLeaveData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceLeave{Libp2PTraceLeave: v} + } + iNdEx = postIndex + case 70: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGraft", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGraft); ok { + if err := oneof.Libp2PTraceGraft.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceGraftData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceGraft{Libp2PTraceGraft: v} + } + iNdEx = postIndex + case 71: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTracePrune", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTracePrune); ok { + if err := oneof.Libp2PTracePrune.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTracePruneData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTracePrune{Libp2PTracePrune: v} + } + iNdEx = postIndex + case 72: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceDuplicateMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceDuplicateMessage); ok { + if err := oneof.Libp2PTraceDuplicateMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceDuplicateMessageData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceDuplicateMessage{Libp2PTraceDuplicateMessage: v} + } + iNdEx = postIndex + case 73: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceDeliverMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceDeliverMessage); ok { + if err := oneof.Libp2PTraceDeliverMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceDeliverMessageData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceDeliverMessage{Libp2PTraceDeliverMessage: v} + } + iNdEx = postIndex + case 74: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTracePublishMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTracePublishMessage); ok { + if err := oneof.Libp2PTracePublishMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTracePublishMessageData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTracePublishMessage{Libp2PTracePublishMessage: v} + } + iNdEx = postIndex + case 75: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRejectMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRejectMessage); ok { + if err := oneof.Libp2PTraceRejectMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRejectMessageData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRejectMessage{Libp2PTraceRejectMessage: v} + } + iNdEx = postIndex + case 76: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlIhave", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlIhave); ok { + if err := oneof.Libp2PTraceRpcMetaControlIhave.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRpcMetaControlIhave{Libp2PTraceRpcMetaControlIhave: v} + } + iNdEx = postIndex + case 77: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlIwant", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlIwant); ok { + if err := oneof.Libp2PTraceRpcMetaControlIwant.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRpcMetaControlIwant{Libp2PTraceRpcMetaControlIwant: v} + } + iNdEx = postIndex + case 78: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlIdontwant", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlIdontwant); ok { + if err := oneof.Libp2PTraceRpcMetaControlIdontwant.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRpcMetaControlIdontwant{Libp2PTraceRpcMetaControlIdontwant: v} + } + iNdEx = postIndex + case 79: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlGraft", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlGraft); ok { + if err := oneof.Libp2PTraceRpcMetaControlGraft.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRpcMetaControlGraft{Libp2PTraceRpcMetaControlGraft: v} + } + iNdEx = postIndex + case 80: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlPrune", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlPrune); ok { + if err := oneof.Libp2PTraceRpcMetaControlPrune.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRpcMetaControlPrune{Libp2PTraceRpcMetaControlPrune: v} + } + iNdEx = postIndex + case 81: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaSubscription", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaSubscription); ok { + if err := oneof.Libp2PTraceRpcMetaSubscription.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRpcMetaSubscription{Libp2PTraceRpcMetaSubscription: v} + } + iNdEx = postIndex + case 82: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaMessage); ok { + if err := oneof.Libp2PTraceRpcMetaMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRpcMetaMessage{Libp2PTraceRpcMetaMessage: v} + } + iNdEx = postIndex + case 83: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecordConsensus", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_NodeRecordConsensus); ok { + if err := oneof.NodeRecordConsensus.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalNodeRecordConsensusData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_NodeRecordConsensus{NodeRecordConsensus: v} + } + iNdEx = postIndex + case 84: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubAggregateAndProof", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubAggregateAndProof); ok { + if err := oneof.Libp2PTraceGossipsubAggregateAndProof.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceGossipsubAggregateAndProof{Libp2PTraceGossipsubAggregateAndProof: v} + } + iNdEx = postIndex + case 85: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsDataColumnSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsDataColumnSidecar); ok { + if err := oneof.EthV1EventsDataColumnSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1EventsDataColumnSidecarData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1EventsDataColumnSidecar{EthV1EventsDataColumnSidecar: v} + } + iNdEx = postIndex + case 86: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubDataColumnSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubDataColumnSidecar); ok { + if err := oneof.Libp2PTraceGossipsubDataColumnSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceGossipsubDataColumnSidecar{Libp2PTraceGossipsubDataColumnSidecar: v} + } + iNdEx = postIndex + case 87: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceSyntheticHeartbeat", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceSyntheticHeartbeat); ok { + if err := oneof.Libp2PTraceSyntheticHeartbeat.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceSyntheticHeartbeat{Libp2PTraceSyntheticHeartbeat: v} + } + iNdEx = postIndex + case 88: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcDataColumnCustodyProbe", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcDataColumnCustodyProbe); ok { + if err := oneof.Libp2PTraceRpcDataColumnCustodyProbe.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceRpcDataColumnCustodyProbe{Libp2PTraceRpcDataColumnCustodyProbe: v} + } + iNdEx = postIndex + case 89: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsensusEngineApiNewPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_ConsensusEngineApiNewPayload); ok { + if err := oneof.ConsensusEngineApiNewPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalConsensusEngineAPINewPayloadData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_ConsensusEngineApiNewPayload{ConsensusEngineApiNewPayload: v} + } + iNdEx = postIndex + case 90: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsensusEngineApiGetBlobs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_ConsensusEngineApiGetBlobs); ok { + if err := oneof.ConsensusEngineApiGetBlobs.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalConsensusEngineAPIGetBlobsData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_ConsensusEngineApiGetBlobs{ConsensusEngineApiGetBlobs: v} + } + iNdEx = postIndex + case 91: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconBlob", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1BeaconBlob); ok { + if err := oneof.EthV1BeaconBlob.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1BeaconBlobData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1BeaconBlob{EthV1BeaconBlob: v} + } + iNdEx = postIndex + case 92: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconSyncCommittee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1BeaconSyncCommittee); ok { + if err := oneof.EthV1BeaconSyncCommittee.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV1BeaconSyncCommitteeData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV1BeaconSyncCommittee{EthV1BeaconSyncCommittee: v} + } + iNdEx = postIndex + case 93: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockSyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockSyncAggregate); ok { + if err := oneof.EthV2BeaconBlockSyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_EthV2BeaconBlockSyncAggregate{EthV2BeaconBlockSyncAggregate: v} + } + iNdEx = postIndex + case 94: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceIdentify", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceIdentify); ok { + if err := oneof.Libp2PTraceIdentify.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ClientMeta_AdditionalLibP2PTraceIdentifyData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ClientMeta_Libp2PTraceIdentify{Libp2PTraceIdentify: v} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_Event) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_Event: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_Event: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceivedDateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ReceivedDateTime == nil { + m.ReceivedDateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.ReceivedDateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_Geo) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_Geo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_Geo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field City", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.City = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Country", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Country = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CountryCode", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CountryCode = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ContinentCode", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ContinentCode = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Latitude", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Latitude = float64(math.Float64frombits(v)) + case 6: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Longitude", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Longitude = float64(math.Float64frombits(v)) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AutonomousSystemNumber", wireType) + } + m.AutonomousSystemNumber = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.AutonomousSystemNumber |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AutonomousSystemOrganization", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AutonomousSystemOrganization = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_Client) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_Client: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_Client: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IP", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IP = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Geo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Geo == nil { + m.Geo = &ServerMeta_Geo{} + } + if err := m.Geo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Group", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Group = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field User", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.User = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_Peer) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_Peer: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_Peer: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Geo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Geo == nil { + m.Geo = &ServerMeta_Geo{} + } + if err := m.Geo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_AdditionalBeaconP2PAttestationData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_AdditionalBeaconP2PAttestationData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Peer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Peer == nil { + m.Peer = &ServerMeta_Peer{} + } + if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_AdditionalLibp2PTraceConnectedData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_AdditionalLibp2PTraceConnectedData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_AdditionalLibp2PTraceConnectedData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Peer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Peer == nil { + m.Peer = &ServerMeta_Peer{} + } + if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_AdditionalLibp2PTraceDisconnectedData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_AdditionalLibp2PTraceDisconnectedData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_AdditionalLibp2PTraceDisconnectedData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Peer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Peer == nil { + m.Peer = &ServerMeta_Peer{} + } + if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Peer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Peer == nil { + m.Peer = &ServerMeta_Peer{} + } + if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_AdditionalLibp2PTraceIdentifyData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_AdditionalLibp2PTraceIdentifyData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_AdditionalLibp2PTraceIdentifyData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Peer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Peer == nil { + m.Peer = &ServerMeta_Peer{} + } + if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_AdditionalNodeRecordConsensusData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_AdditionalNodeRecordConsensusData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_AdditionalNodeRecordConsensusData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Geo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Geo == nil { + m.Geo = &ServerMeta_Geo{} + } + if err := m.Geo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta_AdditionalNodeRecordExecutionData) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta_AdditionalNodeRecordExecutionData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta_AdditionalNodeRecordExecutionData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Geo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Geo == nil { + m.Geo = &ServerMeta_Geo{} + } + if err := m.Geo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ServerMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ServerMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Event == nil { + m.Event = &ServerMeta_Event{} + } + if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Client", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Client == nil { + m.Client = &ServerMeta_Client{} + } + if err := m.Client.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BEACON_P2P_ATTESTATION", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ServerMeta_BEACON_P2P_ATTESTATION); ok { + if err := oneof.BEACON_P2P_ATTESTATION.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ServerMeta_AdditionalBeaconP2PAttestationData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ServerMeta_BEACON_P2P_ATTESTATION{BEACON_P2P_ATTESTATION: v} + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LIBP2P_TRACE_CONNECTED", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ServerMeta_LIBP2P_TRACE_CONNECTED); ok { + if err := oneof.LIBP2P_TRACE_CONNECTED.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ServerMeta_AdditionalLibp2PTraceConnectedData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ServerMeta_LIBP2P_TRACE_CONNECTED{LIBP2P_TRACE_CONNECTED: v} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LIBP2P_TRACE_DISCONNECTED", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ServerMeta_LIBP2P_TRACE_DISCONNECTED); ok { + if err := oneof.LIBP2P_TRACE_DISCONNECTED.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ServerMeta_AdditionalLibp2PTraceDisconnectedData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ServerMeta_LIBP2P_TRACE_DISCONNECTED{LIBP2P_TRACE_DISCONNECTED: v} + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NODE_RECORD_CONSENSUS", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ServerMeta_NODE_RECORD_CONSENSUS); ok { + if err := oneof.NODE_RECORD_CONSENSUS.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ServerMeta_AdditionalNodeRecordConsensusData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ServerMeta_NODE_RECORD_CONSENSUS{NODE_RECORD_CONSENSUS: v} + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NODE_RECORD_EXECUTION", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ServerMeta_NODE_RECORD_EXECUTION); ok { + if err := oneof.NODE_RECORD_EXECUTION.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ServerMeta_AdditionalNodeRecordExecutionData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ServerMeta_NODE_RECORD_EXECUTION{NODE_RECORD_EXECUTION: v} + } + iNdEx = postIndex + case 42: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LIBP2P_TRACE_SYNTHETIC_HEARTBEAT", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ServerMeta_LIBP2P_TRACE_SYNTHETIC_HEARTBEAT); ok { + if err := oneof.LIBP2P_TRACE_SYNTHETIC_HEARTBEAT.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ServerMeta_LIBP2P_TRACE_SYNTHETIC_HEARTBEAT{LIBP2P_TRACE_SYNTHETIC_HEARTBEAT: v} + } + iNdEx = postIndex + case 43: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LIBP2P_TRACE_IDENTIFY", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.AdditionalData.(*ServerMeta_LIBP2P_TRACE_IDENTIFY); ok { + if err := oneof.LIBP2P_TRACE_IDENTIFY.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ServerMeta_AdditionalLibp2PTraceIdentifyData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.AdditionalData = &ServerMeta_LIBP2P_TRACE_IDENTIFY{LIBP2P_TRACE_IDENTIFY: v} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Meta) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Meta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Meta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Client", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Client == nil { + m.Client = &ClientMeta{} + } + if err := m.Client.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Server", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Server == nil { + m.Server = &ServerMeta{} + } + if err := m.Server.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Event) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Event: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Event: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + m.Name = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Name |= Event_Name(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DateTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DateTime == nil { + m.DateTime = ×tamppb1.Timestamp{} + } + if err := (*timestamppb.Timestamp)(m.DateTime).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Id = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionBlockMetrics_StateReads) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionBlockMetrics_StateReads: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionBlockMetrics_StateReads: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Accounts", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Accounts == nil { + m.Accounts = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Accounts).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StorageSlots", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StorageSlots == nil { + m.StorageSlots = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.StorageSlots).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Code", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Code == nil { + m.Code = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Code).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CodeBytes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CodeBytes == nil { + m.CodeBytes = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.CodeBytes).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionBlockMetrics_StateWrites) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionBlockMetrics_StateWrites: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionBlockMetrics_StateWrites: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Accounts", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Accounts == nil { + m.Accounts = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Accounts).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AccountsDeleted", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AccountsDeleted == nil { + m.AccountsDeleted = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.AccountsDeleted).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StorageSlots", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StorageSlots == nil { + m.StorageSlots = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.StorageSlots).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StorageSlotsDeleted", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StorageSlotsDeleted == nil { + m.StorageSlotsDeleted = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.StorageSlotsDeleted).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Code", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Code == nil { + m.Code = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.Code).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CodeBytes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CodeBytes == nil { + m.CodeBytes = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.CodeBytes).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionBlockMetrics_CacheEntry) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionBlockMetrics_CacheEntry: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionBlockMetrics_CacheEntry: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Hits == nil { + m.Hits = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.Hits).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Misses", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Misses == nil { + m.Misses = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.Misses).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HitRate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HitRate == nil { + m.HitRate = &wrapperspb1.DoubleValue{} + } + if err := (*wrapperspb.DoubleValue)(m.HitRate).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionBlockMetrics_CodeCacheEntry) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionBlockMetrics_CodeCacheEntry: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionBlockMetrics_CodeCacheEntry: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Hits == nil { + m.Hits = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.Hits).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Misses", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Misses == nil { + m.Misses = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.Misses).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HitRate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HitRate == nil { + m.HitRate = &wrapperspb1.DoubleValue{} + } + if err := (*wrapperspb.DoubleValue)(m.HitRate).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HitBytes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HitBytes == nil { + m.HitBytes = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.HitBytes).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MissBytes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MissBytes == nil { + m.MissBytes = &wrapperspb1.Int64Value{} + } + if err := (*wrapperspb.Int64Value)(m.MissBytes).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecutionBlockMetrics) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecutionBlockMetrics: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionBlockMetrics: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Source = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockNumber", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockNumber == nil { + m.BlockNumber = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.BlockNumber).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GasUsed == nil { + m.GasUsed = &wrapperspb1.UInt64Value{} + } + if err := (*wrapperspb.UInt64Value)(m.GasUsed).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TxCount", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TxCount == nil { + m.TxCount = &wrapperspb1.UInt32Value{} + } + if err := (*wrapperspb.UInt32Value)(m.TxCount).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecutionMs == nil { + m.ExecutionMs = &wrapperspb1.DoubleValue{} + } + if err := (*wrapperspb.DoubleValue)(m.ExecutionMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateReadMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StateReadMs == nil { + m.StateReadMs = &wrapperspb1.DoubleValue{} + } + if err := (*wrapperspb.DoubleValue)(m.StateReadMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateHashMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StateHashMs == nil { + m.StateHashMs = &wrapperspb1.DoubleValue{} + } + if err := (*wrapperspb.DoubleValue)(m.StateHashMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommitMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CommitMs == nil { + m.CommitMs = &wrapperspb1.DoubleValue{} + } + if err := (*wrapperspb.DoubleValue)(m.CommitMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalMs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TotalMs == nil { + m.TotalMs = &wrapperspb1.DoubleValue{} + } + if err := (*wrapperspb.DoubleValue)(m.TotalMs).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MgasPerSec", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MgasPerSec == nil { + m.MgasPerSec = &wrapperspb1.DoubleValue{} + } + if err := (*wrapperspb.DoubleValue)(m.MgasPerSec).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateReads", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StateReads == nil { + m.StateReads = &ExecutionBlockMetrics_StateReads{} + } + if err := m.StateReads.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateWrites", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StateWrites == nil { + m.StateWrites = &ExecutionBlockMetrics_StateWrites{} + } + if err := m.StateWrites.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AccountCache", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AccountCache == nil { + m.AccountCache = &ExecutionBlockMetrics_CacheEntry{} + } + if err := m.AccountCache.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StorageCache", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StorageCache == nil { + m.StorageCache = &ExecutionBlockMetrics_CacheEntry{} + } + if err := m.StorageCache.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CodeCache", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CodeCache == nil { + m.CodeCache = &ExecutionBlockMetrics_CodeCacheEntry{} + } + if err := m.CodeCache.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DecoratedEvent: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DecoratedEvent: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Event == nil { + m.Event = &Event{} + } + if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Meta", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Meta == nil { + m.Meta = &Meta{} + } + if err := m.Meta.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsAttestation); ok { + if err := oneof.EthV1EventsAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.Attestation{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsAttestation{EthV1EventsAttestation: v} + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsBlock); ok { + if err := oneof.EthV1EventsBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventBlock{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsBlock{EthV1EventsBlock: v} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsChainReorg", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsChainReorg); ok { + if err := oneof.EthV1EventsChainReorg.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventChainReorg{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsChainReorg{EthV1EventsChainReorg: v} + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsFinalizedCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsFinalizedCheckpoint); ok { + if err := oneof.EthV1EventsFinalizedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventFinalizedCheckpoint{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsFinalizedCheckpoint{EthV1EventsFinalizedCheckpoint: v} + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsHead", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsHead); ok { + if err := oneof.EthV1EventsHead.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventHead{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsHead{EthV1EventsHead: v} + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsVoluntaryExit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsVoluntaryExit); ok { + if err := oneof.EthV1EventsVoluntaryExit.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventVoluntaryExit{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsVoluntaryExit{EthV1EventsVoluntaryExit: v} + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsContributionAndProof", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsContributionAndProof); ok { + if err := oneof.EthV1EventsContributionAndProof.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventContributionAndProof{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsContributionAndProof{EthV1EventsContributionAndProof: v} + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MempoolTransaction", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Data = &DecoratedEvent_MempoolTransaction{MempoolTransaction: string(dAtA[iNdEx:postIndex])} + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlock); ok { + if err := oneof.EthV2BeaconBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v2.EventBlock{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlock{EthV2BeaconBlock: v} + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1ForkChoice", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ForkChoice); ok { + if err := oneof.EthV1ForkChoice.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.ForkChoice{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1ForkChoice{EthV1ForkChoice: v} + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1ForkChoiceReorg", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ForkChoiceReorg); ok { + if err := oneof.EthV1ForkChoiceReorg.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &DebugForkChoiceReorg{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1ForkChoiceReorg{EthV1ForkChoiceReorg: v} + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconCommittee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1BeaconCommittee); ok { + if err := oneof.EthV1BeaconCommittee.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.Committee{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1BeaconCommittee{EthV1BeaconCommittee: v} + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1ValidatorAttestationData", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ValidatorAttestationData); ok { + if err := oneof.EthV1ValidatorAttestationData.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.AttestationDataV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1ValidatorAttestationData{EthV1ValidatorAttestationData: v} + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsAttestationV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsAttestationV2); ok { + if err := oneof.EthV1EventsAttestationV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.AttestationV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsAttestationV2{EthV1EventsAttestationV2: v} + } + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsBlockV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsBlockV2); ok { + if err := oneof.EthV1EventsBlockV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventBlockV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsBlockV2{EthV1EventsBlockV2: v} + } + iNdEx = postIndex + case 18: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsChainReorgV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsChainReorgV2); ok { + if err := oneof.EthV1EventsChainReorgV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventChainReorgV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsChainReorgV2{EthV1EventsChainReorgV2: v} + } + iNdEx = postIndex + case 19: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsFinalizedCheckpointV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsFinalizedCheckpointV2); ok { + if err := oneof.EthV1EventsFinalizedCheckpointV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventFinalizedCheckpointV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsFinalizedCheckpointV2{EthV1EventsFinalizedCheckpointV2: v} + } + iNdEx = postIndex + case 20: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsHeadV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsHeadV2); ok { + if err := oneof.EthV1EventsHeadV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventHeadV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsHeadV2{EthV1EventsHeadV2: v} + } + iNdEx = postIndex + case 21: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsVoluntaryExitV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsVoluntaryExitV2); ok { + if err := oneof.EthV1EventsVoluntaryExitV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventVoluntaryExitV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsVoluntaryExitV2{EthV1EventsVoluntaryExitV2: v} + } + iNdEx = postIndex + case 22: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsContributionAndProofV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsContributionAndProofV2); ok { + if err := oneof.EthV1EventsContributionAndProofV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventContributionAndProofV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsContributionAndProofV2{EthV1EventsContributionAndProofV2: v} + } + iNdEx = postIndex + case 23: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MempoolTransactionV2", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Data = &DecoratedEvent_MempoolTransactionV2{MempoolTransactionV2: string(dAtA[iNdEx:postIndex])} + iNdEx = postIndex + case 24: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockV2); ok { + if err := oneof.EthV2BeaconBlockV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v2.EventBlockV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockV2{EthV2BeaconBlockV2: v} + } + iNdEx = postIndex + case 25: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1ForkChoiceV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ForkChoiceV2); ok { + if err := oneof.EthV1ForkChoiceV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.ForkChoiceV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1ForkChoiceV2{EthV1ForkChoiceV2: v} + } + iNdEx = postIndex + case 26: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1ForkChoiceReorgV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ForkChoiceReorgV2); ok { + if err := oneof.EthV1ForkChoiceReorgV2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &DebugForkChoiceReorgV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1ForkChoiceReorgV2{EthV1ForkChoiceReorgV2: v} + } + iNdEx = postIndex + case 27: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockAttesterSlashing", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockAttesterSlashing); ok { + if err := oneof.EthV2BeaconBlockAttesterSlashing.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.AttesterSlashingV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockAttesterSlashing{EthV2BeaconBlockAttesterSlashing: v} + } + iNdEx = postIndex + case 28: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockProposerSlashing", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockProposerSlashing); ok { + if err := oneof.EthV2BeaconBlockProposerSlashing.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.ProposerSlashingV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockProposerSlashing{EthV2BeaconBlockProposerSlashing: v} + } + iNdEx = postIndex + case 29: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockVoluntaryExit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockVoluntaryExit); ok { + if err := oneof.EthV2BeaconBlockVoluntaryExit.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.SignedVoluntaryExitV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockVoluntaryExit{EthV2BeaconBlockVoluntaryExit: v} + } + iNdEx = postIndex + case 30: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockDeposit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockDeposit); ok { + if err := oneof.EthV2BeaconBlockDeposit.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.DepositV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockDeposit{EthV2BeaconBlockDeposit: v} + } + iNdEx = postIndex + case 31: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockBlsToExecutionChange", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockBlsToExecutionChange); ok { + if err := oneof.EthV2BeaconBlockBlsToExecutionChange.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v2.SignedBLSToExecutionChangeV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockBlsToExecutionChange{EthV2BeaconBlockBlsToExecutionChange: v} + } + iNdEx = postIndex + case 32: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockExecutionTransaction", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockExecutionTransaction); ok { + if err := oneof.EthV2BeaconBlockExecutionTransaction.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.Transaction{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockExecutionTransaction{EthV2BeaconBlockExecutionTransaction: v} + } + iNdEx = postIndex + case 33: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockWithdrawal", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockWithdrawal); ok { + if err := oneof.EthV2BeaconBlockWithdrawal.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.WithdrawalV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockWithdrawal{EthV2BeaconBlockWithdrawal: v} + } + iNdEx = postIndex + case 34: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsBlobSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsBlobSidecar); ok { + if err := oneof.EthV1EventsBlobSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventBlobSidecar{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsBlobSidecar{EthV1EventsBlobSidecar: v} + } + iNdEx = postIndex + case 36: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconBlockBlobSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1BeaconBlockBlobSidecar); ok { + if err := oneof.EthV1BeaconBlockBlobSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.BlobSidecar{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1BeaconBlockBlobSidecar{EthV1BeaconBlockBlobSidecar: v} + } + iNdEx = postIndex + case 37: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeaconP2PAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_BeaconP2PAttestation); ok { + if err := oneof.BeaconP2PAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.AttestationV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_BeaconP2PAttestation{BeaconP2PAttestation: v} + } + iNdEx = postIndex + case 38: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1ProposerDuty", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ProposerDuty); ok { + if err := oneof.EthV1ProposerDuty.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.ProposerDuty{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1ProposerDuty{EthV1ProposerDuty: v} + } + iNdEx = postIndex + case 39: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockElaboratedAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockElaboratedAttestation); ok { + if err := oneof.EthV2BeaconBlockElaboratedAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.ElaboratedAttestation{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockElaboratedAttestation{EthV2BeaconBlockElaboratedAttestation: v} + } + iNdEx = postIndex + case 40: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceAddPeer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceAddPeer); ok { + if err := oneof.Libp2PTraceAddPeer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.AddPeer{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceAddPeer{Libp2PTraceAddPeer: v} + } + iNdEx = postIndex + case 41: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRemovePeer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRemovePeer); ok { + if err := oneof.Libp2PTraceRemovePeer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.RemovePeer{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRemovePeer{Libp2PTraceRemovePeer: v} + } + iNdEx = postIndex + case 42: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRecvRpc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRecvRpc); ok { + if err := oneof.Libp2PTraceRecvRpc.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.RecvRPC{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRecvRpc{Libp2PTraceRecvRpc: v} + } + iNdEx = postIndex + case 43: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceSendRpc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceSendRpc); ok { + if err := oneof.Libp2PTraceSendRpc.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.SendRPC{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceSendRpc{Libp2PTraceSendRpc: v} + } + iNdEx = postIndex + case 44: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceJoin", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceJoin); ok { + if err := oneof.Libp2PTraceJoin.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.Join{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceJoin{Libp2PTraceJoin: v} + } + iNdEx = postIndex + case 45: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceConnected", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceConnected); ok { + if err := oneof.Libp2PTraceConnected.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.Connected{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceConnected{Libp2PTraceConnected: v} + } + iNdEx = postIndex + case 46: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceDisconnected", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceDisconnected); ok { + if err := oneof.Libp2PTraceDisconnected.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.Disconnected{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceDisconnected{Libp2PTraceDisconnected: v} + } + iNdEx = postIndex + case 47: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceHandleMetadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceHandleMetadata); ok { + if err := oneof.Libp2PTraceHandleMetadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.HandleMetadata{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceHandleMetadata{Libp2PTraceHandleMetadata: v} + } + iNdEx = postIndex + case 48: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceHandleStatus", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceHandleStatus); ok { + if err := oneof.Libp2PTraceHandleStatus.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.HandleStatus{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceHandleStatus{Libp2PTraceHandleStatus: v} + } + iNdEx = postIndex + case 49: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubBeaconBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubBeaconBlock); ok { + if err := oneof.Libp2PTraceGossipsubBeaconBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &gossipsub.BeaconBlock{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceGossipsubBeaconBlock{Libp2PTraceGossipsubBeaconBlock: v} + } + iNdEx = postIndex + case 50: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubBeaconAttestation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation); ok { + if err := oneof.Libp2PTraceGossipsubBeaconAttestation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.Attestation{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation{Libp2PTraceGossipsubBeaconAttestation: v} + } + iNdEx = postIndex + case 51: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubBlobSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubBlobSidecar); ok { + if err := oneof.Libp2PTraceGossipsubBlobSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &gossipsub.BlobSidecar{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceGossipsubBlobSidecar{Libp2PTraceGossipsubBlobSidecar: v} + } + iNdEx = postIndex + case 52: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1Validators", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1Validators); ok { + if err := oneof.EthV1Validators.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &Validators{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1Validators{EthV1Validators: v} + } + iNdEx = postIndex + case 53: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MevRelayBidTraceBuilderBlockSubmission", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_MevRelayBidTraceBuilderBlockSubmission); ok { + if err := oneof.MevRelayBidTraceBuilderBlockSubmission.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &mevrelay.BidTrace{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_MevRelayBidTraceBuilderBlockSubmission{MevRelayBidTraceBuilderBlockSubmission: v} + } + iNdEx = postIndex + case 54: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MevRelayPayloadDelivered", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_MevRelayPayloadDelivered); ok { + if err := oneof.MevRelayPayloadDelivered.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &mevrelay.ProposerPayloadDelivered{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_MevRelayPayloadDelivered{MevRelayPayloadDelivered: v} + } + iNdEx = postIndex + case 55: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV3ValidatorBlock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV3ValidatorBlock); ok { + if err := oneof.EthV3ValidatorBlock.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v2.EventBlockV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV3ValidatorBlock{EthV3ValidatorBlock: v} + } + iNdEx = postIndex + case 56: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MevRelayValidatorRegistration", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_MevRelayValidatorRegistration); ok { + if err := oneof.MevRelayValidatorRegistration.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &mevrelay.ValidatorRegistration{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_MevRelayValidatorRegistration{MevRelayValidatorRegistration: v} + } + iNdEx = postIndex + case 57: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsBlockGossip", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsBlockGossip); ok { + if err := oneof.EthV1EventsBlockGossip.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventBlockGossip{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsBlockGossip{EthV1EventsBlockGossip: v} + } + iNdEx = postIndex + case 58: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceDropRpc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceDropRpc); ok { + if err := oneof.Libp2PTraceDropRpc.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.DropRPC{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceDropRpc{Libp2PTraceDropRpc: v} + } + iNdEx = postIndex + case 59: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceLeave", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceLeave); ok { + if err := oneof.Libp2PTraceLeave.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.Leave{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceLeave{Libp2PTraceLeave: v} + } + iNdEx = postIndex + case 60: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGraft", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGraft); ok { + if err := oneof.Libp2PTraceGraft.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.Graft{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceGraft{Libp2PTraceGraft: v} + } + iNdEx = postIndex + case 61: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTracePrune", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTracePrune); ok { + if err := oneof.Libp2PTracePrune.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.Prune{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTracePrune{Libp2PTracePrune: v} + } + iNdEx = postIndex + case 62: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceDuplicateMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceDuplicateMessage); ok { + if err := oneof.Libp2PTraceDuplicateMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.DuplicateMessage{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceDuplicateMessage{Libp2PTraceDuplicateMessage: v} + } + iNdEx = postIndex + case 63: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceDeliverMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceDeliverMessage); ok { + if err := oneof.Libp2PTraceDeliverMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.DeliverMessage{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceDeliverMessage{Libp2PTraceDeliverMessage: v} + } + iNdEx = postIndex + case 64: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTracePublishMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTracePublishMessage); ok { + if err := oneof.Libp2PTracePublishMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.PublishMessage{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTracePublishMessage{Libp2PTracePublishMessage: v} + } + iNdEx = postIndex + case 65: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRejectMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRejectMessage); ok { + if err := oneof.Libp2PTraceRejectMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.RejectMessage{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRejectMessage{Libp2PTraceRejectMessage: v} + } + iNdEx = postIndex + case 66: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlIhave", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlIhave); ok { + if err := oneof.Libp2PTraceRpcMetaControlIhave.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.ControlIHaveMetaItem{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRpcMetaControlIhave{Libp2PTraceRpcMetaControlIhave: v} + } + iNdEx = postIndex + case 67: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlIwant", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlIwant); ok { + if err := oneof.Libp2PTraceRpcMetaControlIwant.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.ControlIWantMetaItem{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRpcMetaControlIwant{Libp2PTraceRpcMetaControlIwant: v} + } + iNdEx = postIndex + case 68: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlIdontwant", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlIdontwant); ok { + if err := oneof.Libp2PTraceRpcMetaControlIdontwant.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.ControlIDontWantMetaItem{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRpcMetaControlIdontwant{Libp2PTraceRpcMetaControlIdontwant: v} + } + iNdEx = postIndex + case 69: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlGraft", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlGraft); ok { + if err := oneof.Libp2PTraceRpcMetaControlGraft.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.ControlGraftMetaItem{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRpcMetaControlGraft{Libp2PTraceRpcMetaControlGraft: v} + } + iNdEx = postIndex + case 70: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaControlPrune", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlPrune); ok { + if err := oneof.Libp2PTraceRpcMetaControlPrune.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.ControlPruneMetaItem{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRpcMetaControlPrune{Libp2PTraceRpcMetaControlPrune: v} + } + iNdEx = postIndex + case 71: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaSubscription", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaSubscription); ok { + if err := oneof.Libp2PTraceRpcMetaSubscription.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.SubMetaItem{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRpcMetaSubscription{Libp2PTraceRpcMetaSubscription: v} + } + iNdEx = postIndex + case 72: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcMetaMessage", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaMessage); ok { + if err := oneof.Libp2PTraceRpcMetaMessage.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.MessageMetaItem{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRpcMetaMessage{Libp2PTraceRpcMetaMessage: v} + } + iNdEx = postIndex + case 73: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecordConsensus", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_NodeRecordConsensus); ok { + if err := oneof.NodeRecordConsensus.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &noderecord.Consensus{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_NodeRecordConsensus{NodeRecordConsensus: v} + } + iNdEx = postIndex + case 74: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeRecordExecution", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_NodeRecordExecution); ok { + if err := oneof.NodeRecordExecution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &noderecord.Execution{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_NodeRecordExecution{NodeRecordExecution: v} + } + iNdEx = postIndex + case 75: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubAggregateAndProof", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubAggregateAndProof); ok { + if err := oneof.Libp2PTraceGossipsubAggregateAndProof.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.SignedAggregateAttestationAndProofV2{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceGossipsubAggregateAndProof{Libp2PTraceGossipsubAggregateAndProof: v} + } + iNdEx = postIndex + case 76: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1EventsDataColumnSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsDataColumnSidecar); ok { + if err := oneof.EthV1EventsDataColumnSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.EventDataColumnSidecar{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1EventsDataColumnSidecar{EthV1EventsDataColumnSidecar: v} + } + iNdEx = postIndex + case 77: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceGossipsubDataColumnSidecar", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubDataColumnSidecar); ok { + if err := oneof.Libp2PTraceGossipsubDataColumnSidecar.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &gossipsub.DataColumnSidecar{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceGossipsubDataColumnSidecar{Libp2PTraceGossipsubDataColumnSidecar: v} + } + iNdEx = postIndex + case 78: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceSyntheticHeartbeat", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceSyntheticHeartbeat); ok { + if err := oneof.Libp2PTraceSyntheticHeartbeat.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.SyntheticHeartbeat{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceSyntheticHeartbeat{Libp2PTraceSyntheticHeartbeat: v} + } + iNdEx = postIndex + case 79: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceIdentify", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceIdentify); ok { + if err := oneof.Libp2PTraceIdentify.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.Identify{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceIdentify{Libp2PTraceIdentify: v} + } + iNdEx = postIndex + case 200: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Libp2PTraceRpcDataColumnCustodyProbe", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcDataColumnCustodyProbe); ok { + if err := oneof.Libp2PTraceRpcDataColumnCustodyProbe.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &libp2p.DataColumnCustodyProbe{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_Libp2PTraceRpcDataColumnCustodyProbe{Libp2PTraceRpcDataColumnCustodyProbe: v} + } + iNdEx = postIndex + case 201: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionStateSize", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_ExecutionStateSize); ok { + if err := oneof.ExecutionStateSize.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ExecutionStateSize{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_ExecutionStateSize{ExecutionStateSize: v} + } + iNdEx = postIndex + case 202: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsensusEngineApiNewPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_ConsensusEngineApiNewPayload); ok { + if err := oneof.ConsensusEngineApiNewPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ConsensusEngineAPINewPayload{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_ConsensusEngineApiNewPayload{ConsensusEngineApiNewPayload: v} + } + iNdEx = postIndex + case 203: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsensusEngineApiGetBlobs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_ConsensusEngineApiGetBlobs); ok { + if err := oneof.ConsensusEngineApiGetBlobs.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ConsensusEngineAPIGetBlobs{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_ConsensusEngineApiGetBlobs{ConsensusEngineApiGetBlobs: v} + } + iNdEx = postIndex + case 204: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionEngineNewPayload", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_ExecutionEngineNewPayload); ok { + if err := oneof.ExecutionEngineNewPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ExecutionEngineNewPayload{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_ExecutionEngineNewPayload{ExecutionEngineNewPayload: v} + } + iNdEx = postIndex + case 205: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionEngineGetBlobs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_ExecutionEngineGetBlobs); ok { + if err := oneof.ExecutionEngineGetBlobs.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ExecutionEngineGetBlobs{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_ExecutionEngineGetBlobs{ExecutionEngineGetBlobs: v} + } + iNdEx = postIndex + case 206: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconBlob", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1BeaconBlob); ok { + if err := oneof.EthV1BeaconBlob.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &v1.Blob{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1BeaconBlob{EthV1BeaconBlob: v} + } + iNdEx = postIndex + case 207: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV1BeaconSyncCommittee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1BeaconSyncCommittee); ok { + if err := oneof.EthV1BeaconSyncCommittee.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &SyncCommitteeData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV1BeaconSyncCommittee{EthV1BeaconSyncCommittee: v} + } + iNdEx = postIndex + case 208: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EthV2BeaconBlockSyncAggregate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockSyncAggregate); ok { + if err := oneof.EthV2BeaconBlockSyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &SyncAggregateData{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_EthV2BeaconBlockSyncAggregate{EthV2BeaconBlockSyncAggregate: v} + } + iNdEx = postIndex + case 209: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionBlockMetrics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if oneof, ok := m.Data.(*DecoratedEvent_ExecutionBlockMetrics); ok { + if err := oneof.ExecutionBlockMetrics.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + } else { + v := &ExecutionBlockMetrics{} + if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Data = &DecoratedEvent_ExecutionBlockMetrics{ExecutionBlockMetrics: v} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} From 67c9da42bd99d18134778ffb7ebbfe67e920a962 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 20:05:06 +1000 Subject: [PATCH 10/27] feat(consumoor): use per-topic consumer groups for lag monitoring This change modifies the consumer setup to assign a unique consumer group to each topic stream by appending the topic name to the base consumer group configuration. Consequently, the `LagMonitor` is initialized with a list of these new per-topic consumer groups instead of a single shared group, ensuring accurate lag tracking for individual streams. --- pkg/consumoor/consumoor.go | 6 +++++- pkg/consumoor/source/lag_monitor.go | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/pkg/consumoor/consumoor.go b/pkg/consumoor/consumoor.go index fa147459..ecf20e68 100644 --- a/pkg/consumoor/consumoor.go +++ b/pkg/consumoor/consumoor.go @@ -113,10 +113,14 @@ func New( Info("Discovered Kafka topics for per-topic streams") streams := make([]topicStream, 0, len(topics)) + consumerGroups := make([]string, 0, len(topics)) for _, topic := range topics { topicKafkaCfg := config.Kafka.ApplyTopicOverride(topic) topicKafkaCfg.Topics = []string{"^" + regexp.QuoteMeta(topic) + "$"} + topicKafkaCfg.ConsumerGroup = config.Kafka.ConsumerGroup + "-" + topic + + consumerGroups = append(consumerGroups, topicKafkaCfg.ConsumerGroup) if _, hasOverride := config.Kafka.TopicOverrides[topic]; hasOverride { cLog.WithField("topic", topic). @@ -163,7 +167,7 @@ func New( lagMon, lagErr := source.NewLagMonitor( log, &config.Kafka, - []string{config.Kafka.ConsumerGroup}, + consumerGroups, metrics, ) if lagErr != nil { diff --git a/pkg/consumoor/source/lag_monitor.go b/pkg/consumoor/source/lag_monitor.go index 75b14ff1..df9f0103 100644 --- a/pkg/consumoor/source/lag_monitor.go +++ b/pkg/consumoor/source/lag_monitor.go @@ -31,7 +31,7 @@ type LagMonitor struct { // NewLagMonitor creates a new LagMonitor. Call Start to begin polling. // The consumerGroups slice contains the consumer group names to monitor -// for lag (typically the single base consumer group shared by all streams). +// for lag (one per-topic consumer group). func NewLagMonitor( log logrus.FieldLogger, cfg *KafkaConfig, From b5132b9ecb84a29788dc0e82a0bd9998653c6d0d Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 20:25:25 +1000 Subject: [PATCH 11/27] feat(consumoor): add configurable rebalance timeout for faster partition reassignment Lower the rebalance timeout from Benthos's 45s default to 15s since our OnPartitionsRevoked handler only commits offsets. With cooperative sticky's two-phase rebalance, this reduces worst-case partition reassignment from ~90s to ~30s when scaling. --- example_consumoor.yaml | 1 + pkg/consumoor/config_test.go | 1 + pkg/consumoor/source/benthos.go | 1 + pkg/consumoor/source/benthos_test.go | 4 ++++ pkg/consumoor/source/config.go | 8 ++++++++ pkg/consumoor/source/config_test.go | 1 + 6 files changed, 16 insertions(+) diff --git a/example_consumoor.yaml b/example_consumoor.yaml index 0cd946c2..d2a7fdab 100644 --- a/example_consumoor.yaml +++ b/example_consumoor.yaml @@ -15,6 +15,7 @@ kafka: # maxPartitionFetchBytes: 3145728 # fetchMaxBytes: 10485760 # sessionTimeoutMs: 30000 + # rebalanceTimeout: 15s # max time for members to rejoin during rebalance commitInterval: 5s # Kafka offset commit interval # shutdownTimeout: 30s # max wait for in-flight messages during graceful shutdown # lagPollInterval: 30s # consumer lag poll interval (0 to disable) diff --git a/pkg/consumoor/config_test.go b/pkg/consumoor/config_test.go index 215cacf0..4989a7e1 100644 --- a/pkg/consumoor/config_test.go +++ b/pkg/consumoor/config_test.go @@ -46,6 +46,7 @@ func validKafkaConfig() *source.KafkaConfig { Encoding: "json", OffsetDefault: "earliest", SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, CommitInterval: 5 * time.Second, ShutdownTimeout: 30 * time.Second, MaxInFlight: 64, diff --git a/pkg/consumoor/source/benthos.go b/pkg/consumoor/source/benthos.go index 453ca064..18ed5bd2 100644 --- a/pkg/consumoor/source/benthos.go +++ b/pkg/consumoor/source/benthos.go @@ -118,6 +118,7 @@ func benthosConfigYAML(logLevel string, kafkaConfig *KafkaConfig) ([]byte, error "fetch_max_bytes": fmt.Sprintf("%dB", kafkaConfig.FetchMaxBytes), "session_timeout": fmt.Sprintf("%dms", kafkaConfig.SessionTimeoutMs), "heartbeat_interval": fmt.Sprintf("%dms", kafkaConfig.heartbeatIntervalMs()), + "rebalance_timeout": kafkaConfig.RebalanceTimeout.String(), } if kafkaConfig.TopicRefreshInterval > 0 { diff --git a/pkg/consumoor/source/benthos_test.go b/pkg/consumoor/source/benthos_test.go index 2171085c..5270bd2c 100644 --- a/pkg/consumoor/source/benthos_test.go +++ b/pkg/consumoor/source/benthos_test.go @@ -142,6 +142,7 @@ func TestBenthosConfigYAML(t *testing.T) { MaxPartitionFetchBytes: 1048576, FetchMaxBytes: 5242880, SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, OffsetDefault: "latest", CommitInterval: 7 * time.Second, ConnectTimeout: 10 * time.Second, @@ -187,6 +188,7 @@ func TestBenthosConfigYAML_NoTCPBlockWhenConnectTimeoutZero(t *testing.T) { MaxPartitionFetchBytes: 1048576, FetchMaxBytes: 10485760, SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, OffsetDefault: "earliest", CommitInterval: 5 * time.Second, ConnectTimeout: 0, @@ -573,6 +575,7 @@ func TestKafkaConfig_Validate_OutputBatch(t *testing.T) { MaxPartitionFetchBytes: 1048576, FetchMaxBytes: 10485760, SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, OffsetDefault: "earliest", CommitInterval: 5 * time.Second, ShutdownTimeout: 30 * time.Second, @@ -653,6 +656,7 @@ func TestKafkaConfig_Validate_TopicOverrides(t *testing.T) { MaxPartitionFetchBytes: 1048576, FetchMaxBytes: 10485760, SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, OffsetDefault: "earliest", CommitInterval: 5 * time.Second, ShutdownTimeout: 30 * time.Second, diff --git a/pkg/consumoor/source/config.go b/pkg/consumoor/source/config.go index 3c86cc01..166895d9 100644 --- a/pkg/consumoor/source/config.go +++ b/pkg/consumoor/source/config.go @@ -81,6 +81,10 @@ type KafkaConfig struct { // SessionTimeoutMs is the consumer group session timeout. SessionTimeoutMs int `yaml:"sessionTimeoutMs" default:"30000"` + // RebalanceTimeout is the maximum time group members are allowed to + // take when a rebalance has begun (finish work, commit offsets, rejoin). + // Lower values speed up partition reassignment when scaling. Default: 15s. + RebalanceTimeout time.Duration `yaml:"rebalanceTimeout" default:"15s"` // OffsetDefault controls where to start consuming when no offset exists. // Valid values: "earliest" or "latest". @@ -168,6 +172,10 @@ func (c *KafkaConfig) Validate() error { return errors.New("kafka: sessionTimeoutMs must be > 0") } + if c.RebalanceTimeout < 100*time.Millisecond { + return errors.New("kafka: rebalanceTimeout must be >= 100ms") + } + if c.CommitInterval <= 0 { return errors.New("kafka: commitInterval must be positive") } diff --git a/pkg/consumoor/source/config_test.go b/pkg/consumoor/source/config_test.go index 91b08cb0..eca84526 100644 --- a/pkg/consumoor/source/config_test.go +++ b/pkg/consumoor/source/config_test.go @@ -16,6 +16,7 @@ func validKafkaConfig() KafkaConfig { Encoding: "json", OffsetDefault: "earliest", SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, CommitInterval: 5 * time.Second, ShutdownTimeout: 30 * time.Second, MaxInFlight: 64, From 1e404a85a5ea1059e4a990f79c4f49f0e82d1031 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 25 Feb 2026 21:51:31 +1000 Subject: [PATCH 12/27] perf(consumoor): use sync.Pool for DecoratedEvent via vtprotobuf pool feature Enable vtprotobuf's pool feature to reuse proto message allocations via sync.Pool instead of heap-allocating per-message. UnmarshalVT now acquires nested messages (ClientMeta, ServerMeta, Event, Meta, etc.) from pools, and ResetVT recursively returns them. Event lifecycle is fully synchronous: created in WriteBatch, consumed by FlattenTo which copies data into ch-go column buffers, then returned to pool via defer after WriteBatch completes. No live references escape. --- buf.gen.yaml | 9 +- pkg/consumoor/source/kafka.go | 2 +- pkg/consumoor/source/output.go | 9 + pkg/proto/eth/v1/attestation_vtproto.pb.go | 236 +- pkg/proto/eth/v1/beacon_block_vtproto.pb.go | 680 ++- pkg/proto/eth/v1/blob_vtproto.pb.go | 42 + pkg/proto/eth/v1/checkpoint_vtproto.pb.go | 42 + pkg/proto/eth/v1/committee_vtproto.pb.go | 35 +- pkg/proto/eth/v1/duties_vtproto.pb.go | 21 + pkg/proto/eth/v1/events_vtproto.pb.go | 419 +- .../eth/v1/execution_engine_vtproto.pb.go | 380 +- .../eth/v1/execution_requests_vtproto.pb.go | 126 +- pkg/proto/eth/v1/fork_choice_vtproto.pb.go | 124 +- pkg/proto/eth/v1/sync_committee_vtproto.pb.go | 126 +- pkg/proto/eth/v1/validator_vtproto.pb.go | 45 +- pkg/proto/eth/v2/beacon_block_vtproto.pb.go | 2028 +++++++- pkg/proto/eth/v2/events_vtproto.pb.go | 113 +- pkg/proto/eth/v2/withdrawals_vtproto.pb.go | 125 +- pkg/proto/libp2p/eth_vtproto.pb.go | 42 + pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go | 63 + pkg/proto/libp2p/peer_vtproto.pb.go | 23 + pkg/proto/libp2p/trace_vtproto.pb.go | 958 +++- pkg/proto/mevrelay/bids_vtproto.pb.go | 21 + pkg/proto/mevrelay/payloads_vtproto.pb.go | 21 + pkg/proto/mevrelay/relay_vtproto.pb.go | 66 +- pkg/proto/noderecord/consensus_vtproto.pb.go | 21 + pkg/proto/noderecord/execution_vtproto.pb.go | 21 + pkg/proto/xatu/coordinator_vtproto.pb.go | 1453 +++++- pkg/proto/xatu/event_ingester_vtproto.pb.go | 4546 +++++++++++++++-- 29 files changed, 11100 insertions(+), 697 deletions(-) diff --git a/buf.gen.yaml b/buf.gen.yaml index a61bfc81..12b5ceea 100644 --- a/buf.gen.yaml +++ b/buf.gen.yaml @@ -10,4 +10,11 @@ plugins: out: . opt: - paths=source_relative - - features=marshal+unmarshal+size + - features=marshal+unmarshal+size+pool + - pool=github.com/ethpandaops/xatu/pkg/proto/xatu.* + - pool=github.com/ethpandaops/xatu/pkg/proto/eth/v1.* + - pool=github.com/ethpandaops/xatu/pkg/proto/eth/v2.* + - pool=github.com/ethpandaops/xatu/pkg/proto/libp2p.* + - pool=github.com/ethpandaops/xatu/pkg/proto/libp2p/gossipsub.* + - pool=github.com/ethpandaops/xatu/pkg/proto/mevrelay.* + - pool=github.com/ethpandaops/xatu/pkg/proto/noderecord.* diff --git a/pkg/consumoor/source/kafka.go b/pkg/consumoor/source/kafka.go index 955d8201..e85201fd 100644 --- a/pkg/consumoor/source/kafka.go +++ b/pkg/consumoor/source/kafka.go @@ -94,7 +94,7 @@ func kafkaTopicMetadata(msg *service.Message) string { } func decodeDecoratedEvent(encoding string, data []byte) (*xatu.DecoratedEvent, error) { - event := &xatu.DecoratedEvent{} + event := xatu.DecoratedEventFromVTPool() switch encoding { case "protobuf": diff --git a/pkg/consumoor/source/output.go b/pkg/consumoor/source/output.go index 9c97f311..1cce7a78 100644 --- a/pkg/consumoor/source/output.go +++ b/pkg/consumoor/source/output.go @@ -85,6 +85,13 @@ func (o *xatuClickHouseOutput) WriteBatch( var batchErr *service.BatchError + var pooledEvents []*xatu.DecoratedEvent + defer func() { + for _, ev := range pooledEvents { + ev.ReturnToVTPool() + } + }() + groups := make(map[xatu.Event_Name]*eventGroup, 16) // Phase 1: decode, route, and group by event type. @@ -129,6 +136,8 @@ func (o *xatuClickHouseOutput) WriteBatch( continue } + pooledEvents = append(pooledEvents, event) + outcome := o.router.Route(event) if outcome.Status == router.StatusRejected { diff --git a/pkg/proto/eth/v1/attestation_vtproto.pb.go b/pkg/proto/eth/v1/attestation_vtproto.pb.go index fdd2260f..f1628066 100644 --- a/pkg/proto/eth/v1/attestation_vtproto.pb.go +++ b/pkg/proto/eth/v1/attestation_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -547,6 +548,210 @@ func (m *ElaboratedAttestation) MarshalToSizedBufferVT(dAtA []byte) (int, error) return len(dAtA) - i, nil } +var vtprotoPool_Attestation = sync.Pool{ + New: func() interface{} { + return &Attestation{} + }, +} + +func (m *Attestation) ResetVT() { + if m != nil { + m.Data.ReturnToVTPool() + m.Reset() + } +} +func (m *Attestation) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Attestation.Put(m) + } +} +func AttestationFromVTPool() *Attestation { + return vtprotoPool_Attestation.Get().(*Attestation) +} + +var vtprotoPool_AttestationV2 = sync.Pool{ + New: func() interface{} { + return &AttestationV2{} + }, +} + +func (m *AttestationV2) ResetVT() { + if m != nil { + m.Data.ReturnToVTPool() + m.Reset() + } +} +func (m *AttestationV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AttestationV2.Put(m) + } +} +func AttestationV2FromVTPool() *AttestationV2 { + return vtprotoPool_AttestationV2.Get().(*AttestationV2) +} + +var vtprotoPool_AttestationData = sync.Pool{ + New: func() interface{} { + return &AttestationData{} + }, +} + +func (m *AttestationData) ResetVT() { + if m != nil { + m.Source.ReturnToVTPool() + m.Target.ReturnToVTPool() + m.Reset() + } +} +func (m *AttestationData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AttestationData.Put(m) + } +} +func AttestationDataFromVTPool() *AttestationData { + return vtprotoPool_AttestationData.Get().(*AttestationData) +} + +var vtprotoPool_AttestationDataV2 = sync.Pool{ + New: func() interface{} { + return &AttestationDataV2{} + }, +} + +func (m *AttestationDataV2) ResetVT() { + if m != nil { + m.Source.ReturnToVTPool() + m.Target.ReturnToVTPool() + m.Reset() + } +} +func (m *AttestationDataV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AttestationDataV2.Put(m) + } +} +func AttestationDataV2FromVTPool() *AttestationDataV2 { + return vtprotoPool_AttestationDataV2.Get().(*AttestationDataV2) +} + +var vtprotoPool_AggregateAttestationAndProof = sync.Pool{ + New: func() interface{} { + return &AggregateAttestationAndProof{} + }, +} + +func (m *AggregateAttestationAndProof) ResetVT() { + if m != nil { + m.Aggregate.ReturnToVTPool() + m.Reset() + } +} +func (m *AggregateAttestationAndProof) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AggregateAttestationAndProof.Put(m) + } +} +func AggregateAttestationAndProofFromVTPool() *AggregateAttestationAndProof { + return vtprotoPool_AggregateAttestationAndProof.Get().(*AggregateAttestationAndProof) +} + +var vtprotoPool_SignedAggregateAttestationAndProof = sync.Pool{ + New: func() interface{} { + return &SignedAggregateAttestationAndProof{} + }, +} + +func (m *SignedAggregateAttestationAndProof) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedAggregateAttestationAndProof) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedAggregateAttestationAndProof.Put(m) + } +} +func SignedAggregateAttestationAndProofFromVTPool() *SignedAggregateAttestationAndProof { + return vtprotoPool_SignedAggregateAttestationAndProof.Get().(*SignedAggregateAttestationAndProof) +} + +var vtprotoPool_AggregateAttestationAndProofV2 = sync.Pool{ + New: func() interface{} { + return &AggregateAttestationAndProofV2{} + }, +} + +func (m *AggregateAttestationAndProofV2) ResetVT() { + if m != nil { + m.Aggregate.ReturnToVTPool() + m.Reset() + } +} +func (m *AggregateAttestationAndProofV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AggregateAttestationAndProofV2.Put(m) + } +} +func AggregateAttestationAndProofV2FromVTPool() *AggregateAttestationAndProofV2 { + return vtprotoPool_AggregateAttestationAndProofV2.Get().(*AggregateAttestationAndProofV2) +} + +var vtprotoPool_SignedAggregateAttestationAndProofV2 = sync.Pool{ + New: func() interface{} { + return &SignedAggregateAttestationAndProofV2{} + }, +} + +func (m *SignedAggregateAttestationAndProofV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedAggregateAttestationAndProofV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedAggregateAttestationAndProofV2.Put(m) + } +} +func SignedAggregateAttestationAndProofV2FromVTPool() *SignedAggregateAttestationAndProofV2 { + return vtprotoPool_SignedAggregateAttestationAndProofV2.Get().(*SignedAggregateAttestationAndProofV2) +} + +var vtprotoPool_ElaboratedAttestation = sync.Pool{ + New: func() interface{} { + return &ElaboratedAttestation{} + }, +} + +func (m *ElaboratedAttestation) ResetVT() { + if m != nil { + m.Data.ReturnToVTPool() + for _, mm := range m.ValidatorIndexes { + mm.Reset() + } + f0 := m.ValidatorIndexes[:0] + m.Reset() + m.ValidatorIndexes = f0 + } +} +func (m *ElaboratedAttestation) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ElaboratedAttestation.Put(m) + } +} +func ElaboratedAttestationFromVTPool() *ElaboratedAttestation { + return vtprotoPool_ElaboratedAttestation.Get().(*ElaboratedAttestation) +} func (m *Attestation) SizeVT() (n int) { if m == nil { return 0 @@ -867,7 +1072,7 @@ func (m *Attestation) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Data == nil { - m.Data = &AttestationData{} + m.Data = AttestationDataFromVTPool() } if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1018,7 +1223,7 @@ func (m *AttestationV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Data == nil { - m.Data = &AttestationDataV2{} + m.Data = AttestationDataV2FromVTPool() } if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1175,7 +1380,7 @@ func (m *AttestationData) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Source == nil { - m.Source = &Checkpoint{} + m.Source = CheckpointFromVTPool() } if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1211,7 +1416,7 @@ func (m *AttestationData) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Target == nil { - m.Target = &Checkpoint{} + m.Target = CheckpointFromVTPool() } if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1402,7 +1607,7 @@ func (m *AttestationDataV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Source == nil { - m.Source = &CheckpointV2{} + m.Source = CheckpointV2FromVTPool() } if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1438,7 +1643,7 @@ func (m *AttestationDataV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Target == nil { - m.Target = &CheckpointV2{} + m.Target = CheckpointV2FromVTPool() } if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1576,7 +1781,7 @@ func (m *AggregateAttestationAndProof) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Aggregate == nil { - m.Aggregate = &Attestation{} + m.Aggregate = AttestationFromVTPool() } if err := m.Aggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1663,7 +1868,7 @@ func (m *SignedAggregateAttestationAndProof) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &AggregateAttestationAndProof{} + m.Message = AggregateAttestationAndProofFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1818,7 +2023,7 @@ func (m *AggregateAttestationAndProofV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Aggregate == nil { - m.Aggregate = &AttestationV2{} + m.Aggregate = AttestationV2FromVTPool() } if err := m.Aggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1905,7 +2110,7 @@ func (m *SignedAggregateAttestationAndProofV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &AggregateAttestationAndProofV2{} + m.Message = AggregateAttestationAndProofV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2024,7 +2229,7 @@ func (m *ElaboratedAttestation) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Data == nil { - m.Data = &AttestationDataV2{} + m.Data = AttestationDataV2FromVTPool() } if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2059,7 +2264,14 @@ func (m *ElaboratedAttestation) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ValidatorIndexes = append(m.ValidatorIndexes, &wrapperspb1.UInt64Value{}) + if len(m.ValidatorIndexes) == cap(m.ValidatorIndexes) { + m.ValidatorIndexes = append(m.ValidatorIndexes, &wrapperspb1.UInt64Value{}) + } else { + m.ValidatorIndexes = m.ValidatorIndexes[:len(m.ValidatorIndexes)+1] + if m.ValidatorIndexes[len(m.ValidatorIndexes)-1] == nil { + m.ValidatorIndexes[len(m.ValidatorIndexes)-1] = &wrapperspb1.UInt64Value{} + } + } if err := (*wrapperspb.UInt64Value)(m.ValidatorIndexes[len(m.ValidatorIndexes)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/eth/v1/beacon_block_vtproto.pb.go b/pkg/proto/eth/v1/beacon_block_vtproto.pb.go index 4cbe516b..d261c187 100644 --- a/pkg/proto/eth/v1/beacon_block_vtproto.pb.go +++ b/pkg/proto/eth/v1/beacon_block_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -1485,6 +1486,587 @@ func (m *SyncAggregate) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_BeaconBlock = sync.Pool{ + New: func() interface{} { + return &BeaconBlock{} + }, +} + +func (m *BeaconBlock) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlock) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlock.Put(m) + } +} +func BeaconBlockFromVTPool() *BeaconBlock { + return vtprotoPool_BeaconBlock.Get().(*BeaconBlock) +} + +var vtprotoPool_BeaconBlockV2 = sync.Pool{ + New: func() interface{} { + return &BeaconBlockV2{} + }, +} + +func (m *BeaconBlockV2) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockV2.Put(m) + } +} +func BeaconBlockV2FromVTPool() *BeaconBlockV2 { + return vtprotoPool_BeaconBlockV2.Get().(*BeaconBlockV2) +} + +var vtprotoPool_SignedBeaconBlock = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlock{} + }, +} + +func (m *SignedBeaconBlock) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlock) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlock.Put(m) + } +} +func SignedBeaconBlockFromVTPool() *SignedBeaconBlock { + return vtprotoPool_SignedBeaconBlock.Get().(*SignedBeaconBlock) +} + +var vtprotoPool_SignedBeaconBlockV2 = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockV2{} + }, +} + +func (m *SignedBeaconBlockV2) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockV2.Put(m) + } +} +func SignedBeaconBlockV2FromVTPool() *SignedBeaconBlockV2 { + return vtprotoPool_SignedBeaconBlockV2.Get().(*SignedBeaconBlockV2) +} + +var vtprotoPool_BeaconBlockBody = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBody{} + }, +} + +func (m *BeaconBlockBody) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + } +} +func (m *BeaconBlockBody) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBody.Put(m) + } +} +func BeaconBlockBodyFromVTPool() *BeaconBlockBody { + return vtprotoPool_BeaconBlockBody.Get().(*BeaconBlockBody) +} + +var vtprotoPool_ProposerSlashing = sync.Pool{ + New: func() interface{} { + return &ProposerSlashing{} + }, +} + +func (m *ProposerSlashing) ResetVT() { + if m != nil { + m.SignedHeader_1.ReturnToVTPool() + m.SignedHeader_2.ReturnToVTPool() + m.Reset() + } +} +func (m *ProposerSlashing) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ProposerSlashing.Put(m) + } +} +func ProposerSlashingFromVTPool() *ProposerSlashing { + return vtprotoPool_ProposerSlashing.Get().(*ProposerSlashing) +} + +var vtprotoPool_ProposerSlashingV2 = sync.Pool{ + New: func() interface{} { + return &ProposerSlashingV2{} + }, +} + +func (m *ProposerSlashingV2) ResetVT() { + if m != nil { + m.SignedHeader_1.ReturnToVTPool() + m.SignedHeader_2.ReturnToVTPool() + m.Reset() + } +} +func (m *ProposerSlashingV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ProposerSlashingV2.Put(m) + } +} +func ProposerSlashingV2FromVTPool() *ProposerSlashingV2 { + return vtprotoPool_ProposerSlashingV2.Get().(*ProposerSlashingV2) +} + +var vtprotoPool_AttesterSlashing = sync.Pool{ + New: func() interface{} { + return &AttesterSlashing{} + }, +} + +func (m *AttesterSlashing) ResetVT() { + if m != nil { + m.Attestation_1.ReturnToVTPool() + m.Attestation_2.ReturnToVTPool() + m.Reset() + } +} +func (m *AttesterSlashing) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AttesterSlashing.Put(m) + } +} +func AttesterSlashingFromVTPool() *AttesterSlashing { + return vtprotoPool_AttesterSlashing.Get().(*AttesterSlashing) +} + +var vtprotoPool_AttesterSlashingV2 = sync.Pool{ + New: func() interface{} { + return &AttesterSlashingV2{} + }, +} + +func (m *AttesterSlashingV2) ResetVT() { + if m != nil { + m.Attestation_1.ReturnToVTPool() + m.Attestation_2.ReturnToVTPool() + m.Reset() + } +} +func (m *AttesterSlashingV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AttesterSlashingV2.Put(m) + } +} +func AttesterSlashingV2FromVTPool() *AttesterSlashingV2 { + return vtprotoPool_AttesterSlashingV2.Get().(*AttesterSlashingV2) +} + +var vtprotoPool_Deposit_Data = sync.Pool{ + New: func() interface{} { + return &Deposit_Data{} + }, +} + +func (m *Deposit_Data) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Deposit_Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Deposit_Data.Put(m) + } +} +func Deposit_DataFromVTPool() *Deposit_Data { + return vtprotoPool_Deposit_Data.Get().(*Deposit_Data) +} + +var vtprotoPool_Deposit = sync.Pool{ + New: func() interface{} { + return &Deposit{} + }, +} + +func (m *Deposit) ResetVT() { + if m != nil { + f0 := m.Proof[:0] + m.Data.ReturnToVTPool() + m.Reset() + m.Proof = f0 + } +} +func (m *Deposit) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Deposit.Put(m) + } +} +func DepositFromVTPool() *Deposit { + return vtprotoPool_Deposit.Get().(*Deposit) +} + +var vtprotoPool_DepositV2_Data = sync.Pool{ + New: func() interface{} { + return &DepositV2_Data{} + }, +} + +func (m *DepositV2_Data) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *DepositV2_Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DepositV2_Data.Put(m) + } +} +func DepositV2_DataFromVTPool() *DepositV2_Data { + return vtprotoPool_DepositV2_Data.Get().(*DepositV2_Data) +} + +var vtprotoPool_DepositV2 = sync.Pool{ + New: func() interface{} { + return &DepositV2{} + }, +} + +func (m *DepositV2) ResetVT() { + if m != nil { + f0 := m.Proof[:0] + m.Data.ReturnToVTPool() + m.Reset() + m.Proof = f0 + } +} +func (m *DepositV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DepositV2.Put(m) + } +} +func DepositV2FromVTPool() *DepositV2 { + return vtprotoPool_DepositV2.Get().(*DepositV2) +} + +var vtprotoPool_VoluntaryExit = sync.Pool{ + New: func() interface{} { + return &VoluntaryExit{} + }, +} + +func (m *VoluntaryExit) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *VoluntaryExit) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_VoluntaryExit.Put(m) + } +} +func VoluntaryExitFromVTPool() *VoluntaryExit { + return vtprotoPool_VoluntaryExit.Get().(*VoluntaryExit) +} + +var vtprotoPool_VoluntaryExitV2 = sync.Pool{ + New: func() interface{} { + return &VoluntaryExitV2{} + }, +} + +func (m *VoluntaryExitV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *VoluntaryExitV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_VoluntaryExitV2.Put(m) + } +} +func VoluntaryExitV2FromVTPool() *VoluntaryExitV2 { + return vtprotoPool_VoluntaryExitV2.Get().(*VoluntaryExitV2) +} + +var vtprotoPool_SignedVoluntaryExit = sync.Pool{ + New: func() interface{} { + return &SignedVoluntaryExit{} + }, +} + +func (m *SignedVoluntaryExit) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedVoluntaryExit) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedVoluntaryExit.Put(m) + } +} +func SignedVoluntaryExitFromVTPool() *SignedVoluntaryExit { + return vtprotoPool_SignedVoluntaryExit.Get().(*SignedVoluntaryExit) +} + +var vtprotoPool_SignedVoluntaryExitV2 = sync.Pool{ + New: func() interface{} { + return &SignedVoluntaryExitV2{} + }, +} + +func (m *SignedVoluntaryExitV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedVoluntaryExitV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedVoluntaryExitV2.Put(m) + } +} +func SignedVoluntaryExitV2FromVTPool() *SignedVoluntaryExitV2 { + return vtprotoPool_SignedVoluntaryExitV2.Get().(*SignedVoluntaryExitV2) +} + +var vtprotoPool_Eth1Data = sync.Pool{ + New: func() interface{} { + return &Eth1Data{} + }, +} + +func (m *Eth1Data) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Eth1Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Eth1Data.Put(m) + } +} +func Eth1DataFromVTPool() *Eth1Data { + return vtprotoPool_Eth1Data.Get().(*Eth1Data) +} + +var vtprotoPool_BeaconBlockHeader = sync.Pool{ + New: func() interface{} { + return &BeaconBlockHeader{} + }, +} + +func (m *BeaconBlockHeader) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BeaconBlockHeader) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockHeader.Put(m) + } +} +func BeaconBlockHeaderFromVTPool() *BeaconBlockHeader { + return vtprotoPool_BeaconBlockHeader.Get().(*BeaconBlockHeader) +} + +var vtprotoPool_BeaconBlockHeaderV2 = sync.Pool{ + New: func() interface{} { + return &BeaconBlockHeaderV2{} + }, +} + +func (m *BeaconBlockHeaderV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BeaconBlockHeaderV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockHeaderV2.Put(m) + } +} +func BeaconBlockHeaderV2FromVTPool() *BeaconBlockHeaderV2 { + return vtprotoPool_BeaconBlockHeaderV2.Get().(*BeaconBlockHeaderV2) +} + +var vtprotoPool_SignedBeaconBlockHeader = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockHeader{} + }, +} + +func (m *SignedBeaconBlockHeader) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockHeader) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockHeader.Put(m) + } +} +func SignedBeaconBlockHeaderFromVTPool() *SignedBeaconBlockHeader { + return vtprotoPool_SignedBeaconBlockHeader.Get().(*SignedBeaconBlockHeader) +} + +var vtprotoPool_SignedBeaconBlockHeaderV2 = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockHeaderV2{} + }, +} + +func (m *SignedBeaconBlockHeaderV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockHeaderV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockHeaderV2.Put(m) + } +} +func SignedBeaconBlockHeaderV2FromVTPool() *SignedBeaconBlockHeaderV2 { + return vtprotoPool_SignedBeaconBlockHeaderV2.Get().(*SignedBeaconBlockHeaderV2) +} + +var vtprotoPool_IndexedAttestation = sync.Pool{ + New: func() interface{} { + return &IndexedAttestation{} + }, +} + +func (m *IndexedAttestation) ResetVT() { + if m != nil { + f0 := m.AttestingIndices[:0] + m.Data.ReturnToVTPool() + m.Reset() + m.AttestingIndices = f0 + } +} +func (m *IndexedAttestation) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_IndexedAttestation.Put(m) + } +} +func IndexedAttestationFromVTPool() *IndexedAttestation { + return vtprotoPool_IndexedAttestation.Get().(*IndexedAttestation) +} + +var vtprotoPool_IndexedAttestationV2 = sync.Pool{ + New: func() interface{} { + return &IndexedAttestationV2{} + }, +} + +func (m *IndexedAttestationV2) ResetVT() { + if m != nil { + for _, mm := range m.AttestingIndices { + mm.Reset() + } + f0 := m.AttestingIndices[:0] + m.Data.ReturnToVTPool() + m.Reset() + m.AttestingIndices = f0 + } +} +func (m *IndexedAttestationV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_IndexedAttestationV2.Put(m) + } +} +func IndexedAttestationV2FromVTPool() *IndexedAttestationV2 { + return vtprotoPool_IndexedAttestationV2.Get().(*IndexedAttestationV2) +} + +var vtprotoPool_SyncAggregate = sync.Pool{ + New: func() interface{} { + return &SyncAggregate{} + }, +} + +func (m *SyncAggregate) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *SyncAggregate) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SyncAggregate.Put(m) + } +} +func SyncAggregateFromVTPool() *SyncAggregate { + return vtprotoPool_SyncAggregate.Get().(*SyncAggregate) +} func (m *BeaconBlock) SizeVT() (n int) { if m == nil { return 0 @@ -2207,7 +2789,7 @@ func (m *BeaconBlock) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBody{} + m.Body = BeaconBlockBodyFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2430,7 +3012,7 @@ func (m *BeaconBlockV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBody{} + m.Body = BeaconBlockBodyFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2517,7 +3099,7 @@ func (m *SignedBeaconBlock) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BeaconBlock{} + m.Block = BeaconBlockFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2636,7 +3218,7 @@ func (m *SignedBeaconBlockV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BeaconBlockV2{} + m.Block = BeaconBlockV2FromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2787,7 +3369,7 @@ func (m *BeaconBlockBody) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &Eth1Data{} + m.Eth1Data = Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2854,7 +3436,14 @@ func (m *BeaconBlockBody) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2888,7 +3477,14 @@ func (m *BeaconBlockBody) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2922,7 +3518,14 @@ func (m *BeaconBlockBody) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2956,7 +3559,14 @@ func (m *BeaconBlockBody) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2990,7 +3600,14 @@ func (m *BeaconBlockBody) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -3076,7 +3693,7 @@ func (m *ProposerSlashing) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SignedHeader_1 == nil { - m.SignedHeader_1 = &SignedBeaconBlockHeader{} + m.SignedHeader_1 = SignedBeaconBlockHeaderFromVTPool() } if err := m.SignedHeader_1.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3112,7 +3729,7 @@ func (m *ProposerSlashing) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SignedHeader_2 == nil { - m.SignedHeader_2 = &SignedBeaconBlockHeader{} + m.SignedHeader_2 = SignedBeaconBlockHeaderFromVTPool() } if err := m.SignedHeader_2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3199,7 +3816,7 @@ func (m *ProposerSlashingV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SignedHeader_1 == nil { - m.SignedHeader_1 = &SignedBeaconBlockHeaderV2{} + m.SignedHeader_1 = SignedBeaconBlockHeaderV2FromVTPool() } if err := m.SignedHeader_1.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3235,7 +3852,7 @@ func (m *ProposerSlashingV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SignedHeader_2 == nil { - m.SignedHeader_2 = &SignedBeaconBlockHeaderV2{} + m.SignedHeader_2 = SignedBeaconBlockHeaderV2FromVTPool() } if err := m.SignedHeader_2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3322,7 +3939,7 @@ func (m *AttesterSlashing) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Attestation_1 == nil { - m.Attestation_1 = &IndexedAttestation{} + m.Attestation_1 = IndexedAttestationFromVTPool() } if err := m.Attestation_1.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3358,7 +3975,7 @@ func (m *AttesterSlashing) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Attestation_2 == nil { - m.Attestation_2 = &IndexedAttestation{} + m.Attestation_2 = IndexedAttestationFromVTPool() } if err := m.Attestation_2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3445,7 +4062,7 @@ func (m *AttesterSlashingV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Attestation_1 == nil { - m.Attestation_1 = &IndexedAttestationV2{} + m.Attestation_1 = IndexedAttestationV2FromVTPool() } if err := m.Attestation_1.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3481,7 +4098,7 @@ func (m *AttesterSlashingV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Attestation_2 == nil { - m.Attestation_2 = &IndexedAttestationV2{} + m.Attestation_2 = IndexedAttestationV2FromVTPool() } if err := m.Attestation_2.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3766,7 +4383,7 @@ func (m *Deposit) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Data == nil { - m.Data = &Deposit_Data{} + m.Data = Deposit_DataFromVTPool() } if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4068,7 +4685,7 @@ func (m *DepositV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Data == nil { - m.Data = &DepositV2_Data{} + m.Data = DepositV2_DataFromVTPool() } if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4367,7 +4984,7 @@ func (m *SignedVoluntaryExit) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &VoluntaryExit{} + m.Message = VoluntaryExitFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4486,7 +5103,7 @@ func (m *SignedVoluntaryExitV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &VoluntaryExitV2{} + m.Message = VoluntaryExitV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5143,7 +5760,7 @@ func (m *SignedBeaconBlockHeader) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockHeader{} + m.Message = BeaconBlockHeaderFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5262,7 +5879,7 @@ func (m *SignedBeaconBlockHeaderV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockHeaderV2{} + m.Message = BeaconBlockHeaderV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5403,7 +6020,7 @@ func (m *IndexedAttestation) UnmarshalVT(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.AttestingIndices) == 0 { + if elementCount != 0 && len(m.AttestingIndices) == 0 && cap(m.AttestingIndices) < elementCount { m.AttestingIndices = make([]uint64, 0, elementCount) } for iNdEx < postIndex { @@ -5457,7 +6074,7 @@ func (m *IndexedAttestation) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Data == nil { - m.Data = &AttestationData{} + m.Data = AttestationDataFromVTPool() } if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5575,7 +6192,14 @@ func (m *IndexedAttestationV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttestingIndices = append(m.AttestingIndices, &wrapperspb1.UInt64Value{}) + if len(m.AttestingIndices) == cap(m.AttestingIndices) { + m.AttestingIndices = append(m.AttestingIndices, &wrapperspb1.UInt64Value{}) + } else { + m.AttestingIndices = m.AttestingIndices[:len(m.AttestingIndices)+1] + if m.AttestingIndices[len(m.AttestingIndices)-1] == nil { + m.AttestingIndices[len(m.AttestingIndices)-1] = &wrapperspb1.UInt64Value{} + } + } if err := (*wrapperspb.UInt64Value)(m.AttestingIndices[len(m.AttestingIndices)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -5610,7 +6234,7 @@ func (m *IndexedAttestationV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Data == nil { - m.Data = &AttestationDataV2{} + m.Data = AttestationDataV2FromVTPool() } if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err diff --git a/pkg/proto/eth/v1/blob_vtproto.pb.go b/pkg/proto/eth/v1/blob_vtproto.pb.go index 8667d3fa..3d226771 100644 --- a/pkg/proto/eth/v1/blob_vtproto.pb.go +++ b/pkg/proto/eth/v1/blob_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -209,6 +210,47 @@ func (m *Blob) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_BlobSidecar = sync.Pool{ + New: func() interface{} { + return &BlobSidecar{} + }, +} + +func (m *BlobSidecar) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BlobSidecar) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlobSidecar.Put(m) + } +} +func BlobSidecarFromVTPool() *BlobSidecar { + return vtprotoPool_BlobSidecar.Get().(*BlobSidecar) +} + +var vtprotoPool_Blob = sync.Pool{ + New: func() interface{} { + return &Blob{} + }, +} + +func (m *Blob) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Blob) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Blob.Put(m) + } +} +func BlobFromVTPool() *Blob { + return vtprotoPool_Blob.Get().(*Blob) +} func (m *BlobSidecar) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/eth/v1/checkpoint_vtproto.pb.go b/pkg/proto/eth/v1/checkpoint_vtproto.pb.go index 8fc0369a..db6411f4 100644 --- a/pkg/proto/eth/v1/checkpoint_vtproto.pb.go +++ b/pkg/proto/eth/v1/checkpoint_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -115,6 +116,47 @@ func (m *CheckpointV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_Checkpoint = sync.Pool{ + New: func() interface{} { + return &Checkpoint{} + }, +} + +func (m *Checkpoint) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Checkpoint) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Checkpoint.Put(m) + } +} +func CheckpointFromVTPool() *Checkpoint { + return vtprotoPool_Checkpoint.Get().(*Checkpoint) +} + +var vtprotoPool_CheckpointV2 = sync.Pool{ + New: func() interface{} { + return &CheckpointV2{} + }, +} + +func (m *CheckpointV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *CheckpointV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CheckpointV2.Put(m) + } +} +func CheckpointV2FromVTPool() *CheckpointV2 { + return vtprotoPool_CheckpointV2.Get().(*CheckpointV2) +} func (m *Checkpoint) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/eth/v1/committee_vtproto.pb.go b/pkg/proto/eth/v1/committee_vtproto.pb.go index 6d399eb0..dc9ae78d 100644 --- a/pkg/proto/eth/v1/committee_vtproto.pb.go +++ b/pkg/proto/eth/v1/committee_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -85,6 +86,31 @@ func (m *Committee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_Committee = sync.Pool{ + New: func() interface{} { + return &Committee{} + }, +} + +func (m *Committee) ResetVT() { + if m != nil { + for _, mm := range m.Validators { + mm.Reset() + } + f0 := m.Validators[:0] + m.Reset() + m.Validators = f0 + } +} +func (m *Committee) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Committee.Put(m) + } +} +func CommitteeFromVTPool() *Committee { + return vtprotoPool_Committee.Get().(*Committee) +} func (m *Committee) SizeVT() (n int) { if m == nil { return 0 @@ -239,7 +265,14 @@ func (m *Committee) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + if len(m.Validators) == cap(m.Validators) { + m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + } else { + m.Validators = m.Validators[:len(m.Validators)+1] + if m.Validators[len(m.Validators)-1] == nil { + m.Validators[len(m.Validators)-1] = &wrapperspb1.UInt64Value{} + } + } if err := (*wrapperspb.UInt64Value)(m.Validators[len(m.Validators)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/eth/v1/duties_vtproto.pb.go b/pkg/proto/eth/v1/duties_vtproto.pb.go index cd63dc35..d5313169 100644 --- a/pkg/proto/eth/v1/duties_vtproto.pb.go +++ b/pkg/proto/eth/v1/duties_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -80,6 +81,26 @@ func (m *ProposerDuty) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_ProposerDuty = sync.Pool{ + New: func() interface{} { + return &ProposerDuty{} + }, +} + +func (m *ProposerDuty) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ProposerDuty) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ProposerDuty.Put(m) + } +} +func ProposerDutyFromVTPool() *ProposerDuty { + return vtprotoPool_ProposerDuty.Get().(*ProposerDuty) +} func (m *ProposerDuty) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/eth/v1/events_vtproto.pb.go b/pkg/proto/eth/v1/events_vtproto.pb.go index 4845fc2b..12b19f18 100644 --- a/pkg/proto/eth/v1/events_vtproto.pb.go +++ b/pkg/proto/eth/v1/events_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -1192,6 +1193,412 @@ func (m *EventDataColumnSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error return len(dAtA) - i, nil } +var vtprotoPool_EventHead = sync.Pool{ + New: func() interface{} { + return &EventHead{} + }, +} + +func (m *EventHead) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventHead) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventHead.Put(m) + } +} +func EventHeadFromVTPool() *EventHead { + return vtprotoPool_EventHead.Get().(*EventHead) +} + +var vtprotoPool_EventHeadV2 = sync.Pool{ + New: func() interface{} { + return &EventHeadV2{} + }, +} + +func (m *EventHeadV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventHeadV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventHeadV2.Put(m) + } +} +func EventHeadV2FromVTPool() *EventHeadV2 { + return vtprotoPool_EventHeadV2.Get().(*EventHeadV2) +} + +var vtprotoPool_EventBlock = sync.Pool{ + New: func() interface{} { + return &EventBlock{} + }, +} + +func (m *EventBlock) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventBlock) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventBlock.Put(m) + } +} +func EventBlockFromVTPool() *EventBlock { + return vtprotoPool_EventBlock.Get().(*EventBlock) +} + +var vtprotoPool_EventBlockV2 = sync.Pool{ + New: func() interface{} { + return &EventBlockV2{} + }, +} + +func (m *EventBlockV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventBlockV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventBlockV2.Put(m) + } +} +func EventBlockV2FromVTPool() *EventBlockV2 { + return vtprotoPool_EventBlockV2.Get().(*EventBlockV2) +} + +var vtprotoPool_EventChainReorg = sync.Pool{ + New: func() interface{} { + return &EventChainReorg{} + }, +} + +func (m *EventChainReorg) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventChainReorg) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventChainReorg.Put(m) + } +} +func EventChainReorgFromVTPool() *EventChainReorg { + return vtprotoPool_EventChainReorg.Get().(*EventChainReorg) +} + +var vtprotoPool_EventChainReorgV2 = sync.Pool{ + New: func() interface{} { + return &EventChainReorgV2{} + }, +} + +func (m *EventChainReorgV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventChainReorgV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventChainReorgV2.Put(m) + } +} +func EventChainReorgV2FromVTPool() *EventChainReorgV2 { + return vtprotoPool_EventChainReorgV2.Get().(*EventChainReorgV2) +} + +var vtprotoPool_EventFinalizedCheckpoint = sync.Pool{ + New: func() interface{} { + return &EventFinalizedCheckpoint{} + }, +} + +func (m *EventFinalizedCheckpoint) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventFinalizedCheckpoint) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventFinalizedCheckpoint.Put(m) + } +} +func EventFinalizedCheckpointFromVTPool() *EventFinalizedCheckpoint { + return vtprotoPool_EventFinalizedCheckpoint.Get().(*EventFinalizedCheckpoint) +} + +var vtprotoPool_EventFinalizedCheckpointV2 = sync.Pool{ + New: func() interface{} { + return &EventFinalizedCheckpointV2{} + }, +} + +func (m *EventFinalizedCheckpointV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventFinalizedCheckpointV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventFinalizedCheckpointV2.Put(m) + } +} +func EventFinalizedCheckpointV2FromVTPool() *EventFinalizedCheckpointV2 { + return vtprotoPool_EventFinalizedCheckpointV2.Get().(*EventFinalizedCheckpointV2) +} + +var vtprotoPool_EventVoluntaryExitMessage = sync.Pool{ + New: func() interface{} { + return &EventVoluntaryExitMessage{} + }, +} + +func (m *EventVoluntaryExitMessage) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventVoluntaryExitMessage) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventVoluntaryExitMessage.Put(m) + } +} +func EventVoluntaryExitMessageFromVTPool() *EventVoluntaryExitMessage { + return vtprotoPool_EventVoluntaryExitMessage.Get().(*EventVoluntaryExitMessage) +} + +var vtprotoPool_EventVoluntaryExitMessageV2 = sync.Pool{ + New: func() interface{} { + return &EventVoluntaryExitMessageV2{} + }, +} + +func (m *EventVoluntaryExitMessageV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventVoluntaryExitMessageV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventVoluntaryExitMessageV2.Put(m) + } +} +func EventVoluntaryExitMessageV2FromVTPool() *EventVoluntaryExitMessageV2 { + return vtprotoPool_EventVoluntaryExitMessageV2.Get().(*EventVoluntaryExitMessageV2) +} + +var vtprotoPool_EventVoluntaryExit = sync.Pool{ + New: func() interface{} { + return &EventVoluntaryExit{} + }, +} + +func (m *EventVoluntaryExit) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *EventVoluntaryExit) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventVoluntaryExit.Put(m) + } +} +func EventVoluntaryExitFromVTPool() *EventVoluntaryExit { + return vtprotoPool_EventVoluntaryExit.Get().(*EventVoluntaryExit) +} + +var vtprotoPool_EventVoluntaryExitV2 = sync.Pool{ + New: func() interface{} { + return &EventVoluntaryExitV2{} + }, +} + +func (m *EventVoluntaryExitV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *EventVoluntaryExitV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventVoluntaryExitV2.Put(m) + } +} +func EventVoluntaryExitV2FromVTPool() *EventVoluntaryExitV2 { + return vtprotoPool_EventVoluntaryExitV2.Get().(*EventVoluntaryExitV2) +} + +var vtprotoPool_ContributionAndProof = sync.Pool{ + New: func() interface{} { + return &ContributionAndProof{} + }, +} + +func (m *ContributionAndProof) ResetVT() { + if m != nil { + m.Contribution.ReturnToVTPool() + m.Reset() + } +} +func (m *ContributionAndProof) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ContributionAndProof.Put(m) + } +} +func ContributionAndProofFromVTPool() *ContributionAndProof { + return vtprotoPool_ContributionAndProof.Get().(*ContributionAndProof) +} + +var vtprotoPool_ContributionAndProofV2 = sync.Pool{ + New: func() interface{} { + return &ContributionAndProofV2{} + }, +} + +func (m *ContributionAndProofV2) ResetVT() { + if m != nil { + m.Contribution.ReturnToVTPool() + m.Reset() + } +} +func (m *ContributionAndProofV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ContributionAndProofV2.Put(m) + } +} +func ContributionAndProofV2FromVTPool() *ContributionAndProofV2 { + return vtprotoPool_ContributionAndProofV2.Get().(*ContributionAndProofV2) +} + +var vtprotoPool_EventContributionAndProof = sync.Pool{ + New: func() interface{} { + return &EventContributionAndProof{} + }, +} + +func (m *EventContributionAndProof) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *EventContributionAndProof) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventContributionAndProof.Put(m) + } +} +func EventContributionAndProofFromVTPool() *EventContributionAndProof { + return vtprotoPool_EventContributionAndProof.Get().(*EventContributionAndProof) +} + +var vtprotoPool_EventContributionAndProofV2 = sync.Pool{ + New: func() interface{} { + return &EventContributionAndProofV2{} + }, +} + +func (m *EventContributionAndProofV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *EventContributionAndProofV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventContributionAndProofV2.Put(m) + } +} +func EventContributionAndProofV2FromVTPool() *EventContributionAndProofV2 { + return vtprotoPool_EventContributionAndProofV2.Get().(*EventContributionAndProofV2) +} + +var vtprotoPool_EventBlobSidecar = sync.Pool{ + New: func() interface{} { + return &EventBlobSidecar{} + }, +} + +func (m *EventBlobSidecar) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventBlobSidecar) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventBlobSidecar.Put(m) + } +} +func EventBlobSidecarFromVTPool() *EventBlobSidecar { + return vtprotoPool_EventBlobSidecar.Get().(*EventBlobSidecar) +} + +var vtprotoPool_EventBlockGossip = sync.Pool{ + New: func() interface{} { + return &EventBlockGossip{} + }, +} + +func (m *EventBlockGossip) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EventBlockGossip) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventBlockGossip.Put(m) + } +} +func EventBlockGossipFromVTPool() *EventBlockGossip { + return vtprotoPool_EventBlockGossip.Get().(*EventBlockGossip) +} + +var vtprotoPool_EventDataColumnSidecar = sync.Pool{ + New: func() interface{} { + return &EventDataColumnSidecar{} + }, +} + +func (m *EventDataColumnSidecar) ResetVT() { + if m != nil { + f0 := m.KzgCommitments[:0] + m.Reset() + m.KzgCommitments = f0 + } +} +func (m *EventDataColumnSidecar) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventDataColumnSidecar.Put(m) + } +} +func EventDataColumnSidecarFromVTPool() *EventDataColumnSidecar { + return vtprotoPool_EventDataColumnSidecar.Get().(*EventDataColumnSidecar) +} func (m *EventHead) SizeVT() (n int) { if m == nil { return 0 @@ -3480,7 +3887,7 @@ func (m *EventVoluntaryExit) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &EventVoluntaryExitMessage{} + m.Message = EventVoluntaryExitMessageFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3599,7 +4006,7 @@ func (m *EventVoluntaryExitV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &EventVoluntaryExitMessageV2{} + m.Message = EventVoluntaryExitMessageV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3737,7 +4144,7 @@ func (m *ContributionAndProof) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Contribution == nil { - m.Contribution = &SyncCommitteeContribution{} + m.Contribution = SyncCommitteeContributionFromVTPool() } if err := m.Contribution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3892,7 +4299,7 @@ func (m *ContributionAndProofV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Contribution == nil { - m.Contribution = &SyncCommitteeContributionV2{} + m.Contribution = SyncCommitteeContributionV2FromVTPool() } if err := m.Contribution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4043,7 +4450,7 @@ func (m *EventContributionAndProof) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &ContributionAndProof{} + m.Message = ContributionAndProofFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4162,7 +4569,7 @@ func (m *EventContributionAndProofV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &ContributionAndProofV2{} + m.Message = ContributionAndProofV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err diff --git a/pkg/proto/eth/v1/execution_engine_vtproto.pb.go b/pkg/proto/eth/v1/execution_engine_vtproto.pb.go index b027997a..4a719ad5 100644 --- a/pkg/proto/eth/v1/execution_engine_vtproto.pb.go +++ b/pkg/proto/eth/v1/execution_engine_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -1950,6 +1951,340 @@ func (m *Transaction) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_ExecutionPayload = sync.Pool{ + New: func() interface{} { + return &ExecutionPayload{} + }, +} + +func (m *ExecutionPayload) ResetVT() { + if m != nil { + f0 := m.Transactions[:0] + m.Reset() + m.Transactions = f0 + } +} +func (m *ExecutionPayload) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayload.Put(m) + } +} +func ExecutionPayloadFromVTPool() *ExecutionPayload { + return vtprotoPool_ExecutionPayload.Get().(*ExecutionPayload) +} + +var vtprotoPool_ExecutionPayloadV2 = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadV2{} + }, +} + +func (m *ExecutionPayloadV2) ResetVT() { + if m != nil { + f0 := m.Transactions[:0] + m.Reset() + m.Transactions = f0 + } +} +func (m *ExecutionPayloadV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadV2.Put(m) + } +} +func ExecutionPayloadV2FromVTPool() *ExecutionPayloadV2 { + return vtprotoPool_ExecutionPayloadV2.Get().(*ExecutionPayloadV2) +} + +var vtprotoPool_ExecutionPayloadHeader = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadHeader{} + }, +} + +func (m *ExecutionPayloadHeader) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionPayloadHeader) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadHeader.Put(m) + } +} +func ExecutionPayloadHeaderFromVTPool() *ExecutionPayloadHeader { + return vtprotoPool_ExecutionPayloadHeader.Get().(*ExecutionPayloadHeader) +} + +var vtprotoPool_ExecutionPayloadHeaderV2 = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadHeaderV2{} + }, +} + +func (m *ExecutionPayloadHeaderV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionPayloadHeaderV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadHeaderV2.Put(m) + } +} +func ExecutionPayloadHeaderV2FromVTPool() *ExecutionPayloadHeaderV2 { + return vtprotoPool_ExecutionPayloadHeaderV2.Get().(*ExecutionPayloadHeaderV2) +} + +var vtprotoPool_ExecutionPayloadCapella = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadCapella{} + }, +} + +func (m *ExecutionPayloadCapella) ResetVT() { + if m != nil { + f0 := m.Transactions[:0] + for _, mm := range m.Withdrawals { + mm.ResetVT() + } + f1 := m.Withdrawals[:0] + m.Reset() + m.Transactions = f0 + m.Withdrawals = f1 + } +} +func (m *ExecutionPayloadCapella) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadCapella.Put(m) + } +} +func ExecutionPayloadCapellaFromVTPool() *ExecutionPayloadCapella { + return vtprotoPool_ExecutionPayloadCapella.Get().(*ExecutionPayloadCapella) +} + +var vtprotoPool_ExecutionPayloadCapellaV2 = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadCapellaV2{} + }, +} + +func (m *ExecutionPayloadCapellaV2) ResetVT() { + if m != nil { + f0 := m.Transactions[:0] + for _, mm := range m.Withdrawals { + mm.ResetVT() + } + f1 := m.Withdrawals[:0] + m.Reset() + m.Transactions = f0 + m.Withdrawals = f1 + } +} +func (m *ExecutionPayloadCapellaV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadCapellaV2.Put(m) + } +} +func ExecutionPayloadCapellaV2FromVTPool() *ExecutionPayloadCapellaV2 { + return vtprotoPool_ExecutionPayloadCapellaV2.Get().(*ExecutionPayloadCapellaV2) +} + +var vtprotoPool_ExecutionPayloadDeneb = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadDeneb{} + }, +} + +func (m *ExecutionPayloadDeneb) ResetVT() { + if m != nil { + f0 := m.Transactions[:0] + for _, mm := range m.Withdrawals { + mm.ResetVT() + } + f1 := m.Withdrawals[:0] + m.Reset() + m.Transactions = f0 + m.Withdrawals = f1 + } +} +func (m *ExecutionPayloadDeneb) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadDeneb.Put(m) + } +} +func ExecutionPayloadDenebFromVTPool() *ExecutionPayloadDeneb { + return vtprotoPool_ExecutionPayloadDeneb.Get().(*ExecutionPayloadDeneb) +} + +var vtprotoPool_ExecutionPayloadElectra = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadElectra{} + }, +} + +func (m *ExecutionPayloadElectra) ResetVT() { + if m != nil { + f0 := m.Transactions[:0] + for _, mm := range m.Withdrawals { + mm.ResetVT() + } + f1 := m.Withdrawals[:0] + m.Reset() + m.Transactions = f0 + m.Withdrawals = f1 + } +} +func (m *ExecutionPayloadElectra) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadElectra.Put(m) + } +} +func ExecutionPayloadElectraFromVTPool() *ExecutionPayloadElectra { + return vtprotoPool_ExecutionPayloadElectra.Get().(*ExecutionPayloadElectra) +} + +var vtprotoPool_ExecutionPayloadFulu = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadFulu{} + }, +} + +func (m *ExecutionPayloadFulu) ResetVT() { + if m != nil { + f0 := m.Transactions[:0] + for _, mm := range m.Withdrawals { + mm.ResetVT() + } + f1 := m.Withdrawals[:0] + m.Reset() + m.Transactions = f0 + m.Withdrawals = f1 + } +} +func (m *ExecutionPayloadFulu) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadFulu.Put(m) + } +} +func ExecutionPayloadFuluFromVTPool() *ExecutionPayloadFulu { + return vtprotoPool_ExecutionPayloadFulu.Get().(*ExecutionPayloadFulu) +} + +var vtprotoPool_Withdrawal = sync.Pool{ + New: func() interface{} { + return &Withdrawal{} + }, +} + +func (m *Withdrawal) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Withdrawal) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Withdrawal.Put(m) + } +} +func WithdrawalFromVTPool() *Withdrawal { + return vtprotoPool_Withdrawal.Get().(*Withdrawal) +} + +var vtprotoPool_WithdrawalV2 = sync.Pool{ + New: func() interface{} { + return &WithdrawalV2{} + }, +} + +func (m *WithdrawalV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *WithdrawalV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_WithdrawalV2.Put(m) + } +} +func WithdrawalV2FromVTPool() *WithdrawalV2 { + return vtprotoPool_WithdrawalV2.Get().(*WithdrawalV2) +} + +var vtprotoPool_ExecutionPayloadHeaderCapella = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadHeaderCapella{} + }, +} + +func (m *ExecutionPayloadHeaderCapella) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionPayloadHeaderCapella) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadHeaderCapella.Put(m) + } +} +func ExecutionPayloadHeaderCapellaFromVTPool() *ExecutionPayloadHeaderCapella { + return vtprotoPool_ExecutionPayloadHeaderCapella.Get().(*ExecutionPayloadHeaderCapella) +} + +var vtprotoPool_ExecutionPayloadHeaderCapellaV2 = sync.Pool{ + New: func() interface{} { + return &ExecutionPayloadHeaderCapellaV2{} + }, +} + +func (m *ExecutionPayloadHeaderCapellaV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionPayloadHeaderCapellaV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionPayloadHeaderCapellaV2.Put(m) + } +} +func ExecutionPayloadHeaderCapellaV2FromVTPool() *ExecutionPayloadHeaderCapellaV2 { + return vtprotoPool_ExecutionPayloadHeaderCapellaV2.Get().(*ExecutionPayloadHeaderCapellaV2) +} + +var vtprotoPool_Transaction = sync.Pool{ + New: func() interface{} { + return &Transaction{} + }, +} + +func (m *Transaction) ResetVT() { + if m != nil { + f0 := m.BlobHashes[:0] + m.Reset() + m.BlobHashes = f0 + } +} +func (m *Transaction) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Transaction.Put(m) + } +} +func TransactionFromVTPool() *Transaction { + return vtprotoPool_Transaction.Get().(*Transaction) +} func (m *ExecutionPayload) SizeVT() (n int) { if m == nil { return 0 @@ -5235,7 +5570,14 @@ func (m *ExecutionPayloadCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Withdrawals = append(m.Withdrawals, &Withdrawal{}) + if len(m.Withdrawals) == cap(m.Withdrawals) { + m.Withdrawals = append(m.Withdrawals, &Withdrawal{}) + } else { + m.Withdrawals = m.Withdrawals[:len(m.Withdrawals)+1] + if m.Withdrawals[len(m.Withdrawals)-1] == nil { + m.Withdrawals[len(m.Withdrawals)-1] = &Withdrawal{} + } + } if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -5784,7 +6126,14 @@ func (m *ExecutionPayloadCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + if len(m.Withdrawals) == cap(m.Withdrawals) { + m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + } else { + m.Withdrawals = m.Withdrawals[:len(m.Withdrawals)+1] + if m.Withdrawals[len(m.Withdrawals)-1] == nil { + m.Withdrawals[len(m.Withdrawals)-1] = &WithdrawalV2{} + } + } if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -6333,7 +6682,14 @@ func (m *ExecutionPayloadDeneb) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + if len(m.Withdrawals) == cap(m.Withdrawals) { + m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + } else { + m.Withdrawals = m.Withdrawals[:len(m.Withdrawals)+1] + if m.Withdrawals[len(m.Withdrawals)-1] == nil { + m.Withdrawals[len(m.Withdrawals)-1] = &WithdrawalV2{} + } + } if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -6954,7 +7310,14 @@ func (m *ExecutionPayloadElectra) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + if len(m.Withdrawals) == cap(m.Withdrawals) { + m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + } else { + m.Withdrawals = m.Withdrawals[:len(m.Withdrawals)+1] + if m.Withdrawals[len(m.Withdrawals)-1] == nil { + m.Withdrawals[len(m.Withdrawals)-1] = &WithdrawalV2{} + } + } if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -7575,7 +7938,14 @@ func (m *ExecutionPayloadFulu) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + if len(m.Withdrawals) == cap(m.Withdrawals) { + m.Withdrawals = append(m.Withdrawals, &WithdrawalV2{}) + } else { + m.Withdrawals = m.Withdrawals[:len(m.Withdrawals)+1] + if m.Withdrawals[len(m.Withdrawals)-1] == nil { + m.Withdrawals[len(m.Withdrawals)-1] = &WithdrawalV2{} + } + } if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/eth/v1/execution_requests_vtproto.pb.go b/pkg/proto/eth/v1/execution_requests_vtproto.pb.go index 1e87ae65..d1ee8418 100644 --- a/pkg/proto/eth/v1/execution_requests_vtproto.pb.go +++ b/pkg/proto/eth/v1/execution_requests_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -298,6 +299,104 @@ func (m *ElectraExecutionRequestConsolidation) MarshalToSizedBufferVT(dAtA []byt return len(dAtA) - i, nil } +var vtprotoPool_ElectraExecutionRequests = sync.Pool{ + New: func() interface{} { + return &ElectraExecutionRequests{} + }, +} + +func (m *ElectraExecutionRequests) ResetVT() { + if m != nil { + for _, mm := range m.Deposits { + mm.ResetVT() + } + f0 := m.Deposits[:0] + for _, mm := range m.Withdrawals { + mm.ResetVT() + } + f1 := m.Withdrawals[:0] + for _, mm := range m.Consolidations { + mm.ResetVT() + } + f2 := m.Consolidations[:0] + m.Reset() + m.Deposits = f0 + m.Withdrawals = f1 + m.Consolidations = f2 + } +} +func (m *ElectraExecutionRequests) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ElectraExecutionRequests.Put(m) + } +} +func ElectraExecutionRequestsFromVTPool() *ElectraExecutionRequests { + return vtprotoPool_ElectraExecutionRequests.Get().(*ElectraExecutionRequests) +} + +var vtprotoPool_ElectraExecutionRequestDeposit = sync.Pool{ + New: func() interface{} { + return &ElectraExecutionRequestDeposit{} + }, +} + +func (m *ElectraExecutionRequestDeposit) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ElectraExecutionRequestDeposit) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ElectraExecutionRequestDeposit.Put(m) + } +} +func ElectraExecutionRequestDepositFromVTPool() *ElectraExecutionRequestDeposit { + return vtprotoPool_ElectraExecutionRequestDeposit.Get().(*ElectraExecutionRequestDeposit) +} + +var vtprotoPool_ElectraExecutionRequestWithdrawal = sync.Pool{ + New: func() interface{} { + return &ElectraExecutionRequestWithdrawal{} + }, +} + +func (m *ElectraExecutionRequestWithdrawal) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ElectraExecutionRequestWithdrawal) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ElectraExecutionRequestWithdrawal.Put(m) + } +} +func ElectraExecutionRequestWithdrawalFromVTPool() *ElectraExecutionRequestWithdrawal { + return vtprotoPool_ElectraExecutionRequestWithdrawal.Get().(*ElectraExecutionRequestWithdrawal) +} + +var vtprotoPool_ElectraExecutionRequestConsolidation = sync.Pool{ + New: func() interface{} { + return &ElectraExecutionRequestConsolidation{} + }, +} + +func (m *ElectraExecutionRequestConsolidation) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ElectraExecutionRequestConsolidation) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ElectraExecutionRequestConsolidation.Put(m) + } +} +func ElectraExecutionRequestConsolidationFromVTPool() *ElectraExecutionRequestConsolidation { + return vtprotoPool_ElectraExecutionRequestConsolidation.Get().(*ElectraExecutionRequestConsolidation) +} func (m *ElectraExecutionRequests) SizeVT() (n int) { if m == nil { return 0 @@ -458,7 +557,14 @@ func (m *ElectraExecutionRequests) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &ElectraExecutionRequestDeposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &ElectraExecutionRequestDeposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &ElectraExecutionRequestDeposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -492,7 +598,14 @@ func (m *ElectraExecutionRequests) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Withdrawals = append(m.Withdrawals, &ElectraExecutionRequestWithdrawal{}) + if len(m.Withdrawals) == cap(m.Withdrawals) { + m.Withdrawals = append(m.Withdrawals, &ElectraExecutionRequestWithdrawal{}) + } else { + m.Withdrawals = m.Withdrawals[:len(m.Withdrawals)+1] + if m.Withdrawals[len(m.Withdrawals)-1] == nil { + m.Withdrawals[len(m.Withdrawals)-1] = &ElectraExecutionRequestWithdrawal{} + } + } if err := m.Withdrawals[len(m.Withdrawals)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -526,7 +639,14 @@ func (m *ElectraExecutionRequests) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Consolidations = append(m.Consolidations, &ElectraExecutionRequestConsolidation{}) + if len(m.Consolidations) == cap(m.Consolidations) { + m.Consolidations = append(m.Consolidations, &ElectraExecutionRequestConsolidation{}) + } else { + m.Consolidations = m.Consolidations[:len(m.Consolidations)+1] + if m.Consolidations[len(m.Consolidations)-1] == nil { + m.Consolidations[len(m.Consolidations)-1] = &ElectraExecutionRequestConsolidation{} + } + } if err := m.Consolidations[len(m.Consolidations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/eth/v1/fork_choice_vtproto.pb.go b/pkg/proto/eth/v1/fork_choice_vtproto.pb.go index 1ed0f1aa..28c4e80d 100644 --- a/pkg/proto/eth/v1/fork_choice_vtproto.pb.go +++ b/pkg/proto/eth/v1/fork_choice_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -346,6 +347,103 @@ func (m *ForkChoiceNodeV2) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_ForkChoice = sync.Pool{ + New: func() interface{} { + return &ForkChoice{} + }, +} + +func (m *ForkChoice) ResetVT() { + if m != nil { + m.JustifiedCheckpoint.ReturnToVTPool() + m.FinalizedCheckpoint.ReturnToVTPool() + for _, mm := range m.ForkChoiceNodes { + mm.ResetVT() + } + f0 := m.ForkChoiceNodes[:0] + m.Reset() + m.ForkChoiceNodes = f0 + } +} +func (m *ForkChoice) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ForkChoice.Put(m) + } +} +func ForkChoiceFromVTPool() *ForkChoice { + return vtprotoPool_ForkChoice.Get().(*ForkChoice) +} + +var vtprotoPool_ForkChoiceV2 = sync.Pool{ + New: func() interface{} { + return &ForkChoiceV2{} + }, +} + +func (m *ForkChoiceV2) ResetVT() { + if m != nil { + m.JustifiedCheckpoint.ReturnToVTPool() + m.FinalizedCheckpoint.ReturnToVTPool() + for _, mm := range m.ForkChoiceNodes { + mm.ResetVT() + } + f0 := m.ForkChoiceNodes[:0] + m.Reset() + m.ForkChoiceNodes = f0 + } +} +func (m *ForkChoiceV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ForkChoiceV2.Put(m) + } +} +func ForkChoiceV2FromVTPool() *ForkChoiceV2 { + return vtprotoPool_ForkChoiceV2.Get().(*ForkChoiceV2) +} + +var vtprotoPool_ForkChoiceNode = sync.Pool{ + New: func() interface{} { + return &ForkChoiceNode{} + }, +} + +func (m *ForkChoiceNode) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ForkChoiceNode) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ForkChoiceNode.Put(m) + } +} +func ForkChoiceNodeFromVTPool() *ForkChoiceNode { + return vtprotoPool_ForkChoiceNode.Get().(*ForkChoiceNode) +} + +var vtprotoPool_ForkChoiceNodeV2 = sync.Pool{ + New: func() interface{} { + return &ForkChoiceNodeV2{} + }, +} + +func (m *ForkChoiceNodeV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ForkChoiceNodeV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ForkChoiceNodeV2.Put(m) + } +} +func ForkChoiceNodeV2FromVTPool() *ForkChoiceNodeV2 { + return vtprotoPool_ForkChoiceNodeV2.Get().(*ForkChoiceNodeV2) +} func (m *ForkChoice) SizeVT() (n int) { if m == nil { return 0 @@ -541,7 +639,7 @@ func (m *ForkChoice) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.JustifiedCheckpoint == nil { - m.JustifiedCheckpoint = &Checkpoint{} + m.JustifiedCheckpoint = CheckpointFromVTPool() } if err := m.JustifiedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -577,7 +675,7 @@ func (m *ForkChoice) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.FinalizedCheckpoint == nil { - m.FinalizedCheckpoint = &Checkpoint{} + m.FinalizedCheckpoint = CheckpointFromVTPool() } if err := m.FinalizedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -612,7 +710,14 @@ func (m *ForkChoice) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ForkChoiceNodes = append(m.ForkChoiceNodes, &ForkChoiceNode{}) + if len(m.ForkChoiceNodes) == cap(m.ForkChoiceNodes) { + m.ForkChoiceNodes = append(m.ForkChoiceNodes, &ForkChoiceNode{}) + } else { + m.ForkChoiceNodes = m.ForkChoiceNodes[:len(m.ForkChoiceNodes)+1] + if m.ForkChoiceNodes[len(m.ForkChoiceNodes)-1] == nil { + m.ForkChoiceNodes[len(m.ForkChoiceNodes)-1] = &ForkChoiceNode{} + } + } if err := m.ForkChoiceNodes[len(m.ForkChoiceNodes)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -698,7 +803,7 @@ func (m *ForkChoiceV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.JustifiedCheckpoint == nil { - m.JustifiedCheckpoint = &CheckpointV2{} + m.JustifiedCheckpoint = CheckpointV2FromVTPool() } if err := m.JustifiedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -734,7 +839,7 @@ func (m *ForkChoiceV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.FinalizedCheckpoint == nil { - m.FinalizedCheckpoint = &CheckpointV2{} + m.FinalizedCheckpoint = CheckpointV2FromVTPool() } if err := m.FinalizedCheckpoint.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -769,7 +874,14 @@ func (m *ForkChoiceV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ForkChoiceNodes = append(m.ForkChoiceNodes, &ForkChoiceNodeV2{}) + if len(m.ForkChoiceNodes) == cap(m.ForkChoiceNodes) { + m.ForkChoiceNodes = append(m.ForkChoiceNodes, &ForkChoiceNodeV2{}) + } else { + m.ForkChoiceNodes = m.ForkChoiceNodes[:len(m.ForkChoiceNodes)+1] + if m.ForkChoiceNodes[len(m.ForkChoiceNodes)-1] == nil { + m.ForkChoiceNodes[len(m.ForkChoiceNodes)-1] = &ForkChoiceNodeV2{} + } + } if err := m.ForkChoiceNodes[len(m.ForkChoiceNodes)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/eth/v1/sync_committee_vtproto.pb.go b/pkg/proto/eth/v1/sync_committee_vtproto.pb.go index fe64497b..9272d814 100644 --- a/pkg/proto/eth/v1/sync_committee_vtproto.pb.go +++ b/pkg/proto/eth/v1/sync_committee_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -260,6 +261,104 @@ func (m *SyncCommittee) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_SyncCommitteeContribution = sync.Pool{ + New: func() interface{} { + return &SyncCommitteeContribution{} + }, +} + +func (m *SyncCommitteeContribution) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *SyncCommitteeContribution) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SyncCommitteeContribution.Put(m) + } +} +func SyncCommitteeContributionFromVTPool() *SyncCommitteeContribution { + return vtprotoPool_SyncCommitteeContribution.Get().(*SyncCommitteeContribution) +} + +var vtprotoPool_SyncCommitteeContributionV2 = sync.Pool{ + New: func() interface{} { + return &SyncCommitteeContributionV2{} + }, +} + +func (m *SyncCommitteeContributionV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *SyncCommitteeContributionV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SyncCommitteeContributionV2.Put(m) + } +} +func SyncCommitteeContributionV2FromVTPool() *SyncCommitteeContributionV2 { + return vtprotoPool_SyncCommitteeContributionV2.Get().(*SyncCommitteeContributionV2) +} + +var vtprotoPool_SyncCommitteeValidatorAggregate = sync.Pool{ + New: func() interface{} { + return &SyncCommitteeValidatorAggregate{} + }, +} + +func (m *SyncCommitteeValidatorAggregate) ResetVT() { + if m != nil { + for _, mm := range m.Validators { + mm.Reset() + } + f0 := m.Validators[:0] + m.Reset() + m.Validators = f0 + } +} +func (m *SyncCommitteeValidatorAggregate) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SyncCommitteeValidatorAggregate.Put(m) + } +} +func SyncCommitteeValidatorAggregateFromVTPool() *SyncCommitteeValidatorAggregate { + return vtprotoPool_SyncCommitteeValidatorAggregate.Get().(*SyncCommitteeValidatorAggregate) +} + +var vtprotoPool_SyncCommittee = sync.Pool{ + New: func() interface{} { + return &SyncCommittee{} + }, +} + +func (m *SyncCommittee) ResetVT() { + if m != nil { + for _, mm := range m.Validators { + mm.Reset() + } + f0 := m.Validators[:0] + for _, mm := range m.ValidatorAggregates { + mm.ResetVT() + } + f1 := m.ValidatorAggregates[:0] + m.Reset() + m.Validators = f0 + m.ValidatorAggregates = f1 + } +} +func (m *SyncCommittee) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SyncCommittee.Put(m) + } +} +func SyncCommitteeFromVTPool() *SyncCommittee { + return vtprotoPool_SyncCommittee.Get().(*SyncCommittee) +} func (m *SyncCommitteeContribution) SizeVT() (n int) { if m == nil { return 0 @@ -818,7 +917,14 @@ func (m *SyncCommitteeValidatorAggregate) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + if len(m.Validators) == cap(m.Validators) { + m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + } else { + m.Validators = m.Validators[:len(m.Validators)+1] + if m.Validators[len(m.Validators)-1] == nil { + m.Validators[len(m.Validators)-1] = &wrapperspb1.UInt64Value{} + } + } if err := (*wrapperspb.UInt64Value)(m.Validators[len(m.Validators)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -903,7 +1009,14 @@ func (m *SyncCommittee) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + if len(m.Validators) == cap(m.Validators) { + m.Validators = append(m.Validators, &wrapperspb1.UInt64Value{}) + } else { + m.Validators = m.Validators[:len(m.Validators)+1] + if m.Validators[len(m.Validators)-1] == nil { + m.Validators[len(m.Validators)-1] = &wrapperspb1.UInt64Value{} + } + } if err := (*wrapperspb.UInt64Value)(m.Validators[len(m.Validators)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -937,7 +1050,14 @@ func (m *SyncCommittee) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ValidatorAggregates = append(m.ValidatorAggregates, &SyncCommitteeValidatorAggregate{}) + if len(m.ValidatorAggregates) == cap(m.ValidatorAggregates) { + m.ValidatorAggregates = append(m.ValidatorAggregates, &SyncCommitteeValidatorAggregate{}) + } else { + m.ValidatorAggregates = m.ValidatorAggregates[:len(m.ValidatorAggregates)+1] + if m.ValidatorAggregates[len(m.ValidatorAggregates)-1] == nil { + m.ValidatorAggregates[len(m.ValidatorAggregates)-1] = &SyncCommitteeValidatorAggregate{} + } + } if err := m.ValidatorAggregates[len(m.ValidatorAggregates)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/eth/v1/validator_vtproto.pb.go b/pkg/proto/eth/v1/validator_vtproto.pb.go index 95f26be9..51896afb 100644 --- a/pkg/proto/eth/v1/validator_vtproto.pb.go +++ b/pkg/proto/eth/v1/validator_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -206,6 +207,48 @@ func (m *Validator) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_ValidatorData = sync.Pool{ + New: func() interface{} { + return &ValidatorData{} + }, +} + +func (m *ValidatorData) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ValidatorData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ValidatorData.Put(m) + } +} +func ValidatorDataFromVTPool() *ValidatorData { + return vtprotoPool_ValidatorData.Get().(*ValidatorData) +} + +var vtprotoPool_Validator = sync.Pool{ + New: func() interface{} { + return &Validator{} + }, +} + +func (m *Validator) ResetVT() { + if m != nil { + m.Data.ReturnToVTPool() + m.Reset() + } +} +func (m *Validator) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Validator.Put(m) + } +} +func ValidatorFromVTPool() *Validator { + return vtprotoPool_Validator.Get().(*Validator) +} func (m *ValidatorData) SizeVT() (n int) { if m == nil { return 0 @@ -672,7 +715,7 @@ func (m *Validator) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Data == nil { - m.Data = &ValidatorData{} + m.Data = ValidatorDataFromVTPool() } if err := m.Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err diff --git a/pkg/proto/eth/v2/beacon_block_vtproto.pb.go b/pkg/proto/eth/v2/beacon_block_vtproto.pb.go index 6d48d793..98bfb938 100644 --- a/pkg/proto/eth/v2/beacon_block_vtproto.pb.go +++ b/pkg/proto/eth/v2/beacon_block_vtproto.pb.go @@ -12,6 +12,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -3514,6 +3515,1255 @@ func (m *BeaconBlockBodyAltairV2) MarshalToSizedBufferVT(dAtA []byte) (int, erro return len(dAtA) - i, nil } +var vtprotoPool_SignedBeaconBlockBellatrix = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockBellatrix{} + }, +} + +func (m *SignedBeaconBlockBellatrix) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockBellatrix) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockBellatrix.Put(m) + } +} +func SignedBeaconBlockBellatrixFromVTPool() *SignedBeaconBlockBellatrix { + return vtprotoPool_SignedBeaconBlockBellatrix.Get().(*SignedBeaconBlockBellatrix) +} + +var vtprotoPool_SignedBeaconBlockBellatrixV2 = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockBellatrixV2{} + }, +} + +func (m *SignedBeaconBlockBellatrixV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockBellatrixV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockBellatrixV2.Put(m) + } +} +func SignedBeaconBlockBellatrixV2FromVTPool() *SignedBeaconBlockBellatrixV2 { + return vtprotoPool_SignedBeaconBlockBellatrixV2.Get().(*SignedBeaconBlockBellatrixV2) +} + +var vtprotoPool_SignedBeaconBlockCapella = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockCapella{} + }, +} + +func (m *SignedBeaconBlockCapella) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockCapella) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockCapella.Put(m) + } +} +func SignedBeaconBlockCapellaFromVTPool() *SignedBeaconBlockCapella { + return vtprotoPool_SignedBeaconBlockCapella.Get().(*SignedBeaconBlockCapella) +} + +var vtprotoPool_SignedBeaconBlockCapellaV2 = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockCapellaV2{} + }, +} + +func (m *SignedBeaconBlockCapellaV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockCapellaV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockCapellaV2.Put(m) + } +} +func SignedBeaconBlockCapellaV2FromVTPool() *SignedBeaconBlockCapellaV2 { + return vtprotoPool_SignedBeaconBlockCapellaV2.Get().(*SignedBeaconBlockCapellaV2) +} + +var vtprotoPool_SignedBeaconBlockDeneb = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockDeneb{} + }, +} + +func (m *SignedBeaconBlockDeneb) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockDeneb) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockDeneb.Put(m) + } +} +func SignedBeaconBlockDenebFromVTPool() *SignedBeaconBlockDeneb { + return vtprotoPool_SignedBeaconBlockDeneb.Get().(*SignedBeaconBlockDeneb) +} + +var vtprotoPool_SignedBeaconBlockElectra = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockElectra{} + }, +} + +func (m *SignedBeaconBlockElectra) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockElectra) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockElectra.Put(m) + } +} +func SignedBeaconBlockElectraFromVTPool() *SignedBeaconBlockElectra { + return vtprotoPool_SignedBeaconBlockElectra.Get().(*SignedBeaconBlockElectra) +} + +var vtprotoPool_SignedBeaconBlockFulu = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockFulu{} + }, +} + +func (m *SignedBeaconBlockFulu) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockFulu) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockFulu.Put(m) + } +} +func SignedBeaconBlockFuluFromVTPool() *SignedBeaconBlockFulu { + return vtprotoPool_SignedBeaconBlockFulu.Get().(*SignedBeaconBlockFulu) +} + +var vtprotoPool_SignedBlindedBeaconBlockBellatrix = sync.Pool{ + New: func() interface{} { + return &SignedBlindedBeaconBlockBellatrix{} + }, +} + +func (m *SignedBlindedBeaconBlockBellatrix) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBlindedBeaconBlockBellatrix) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBlindedBeaconBlockBellatrix.Put(m) + } +} +func SignedBlindedBeaconBlockBellatrixFromVTPool() *SignedBlindedBeaconBlockBellatrix { + return vtprotoPool_SignedBlindedBeaconBlockBellatrix.Get().(*SignedBlindedBeaconBlockBellatrix) +} + +var vtprotoPool_SignedBlindedBeaconBlockBellatrixV2 = sync.Pool{ + New: func() interface{} { + return &SignedBlindedBeaconBlockBellatrixV2{} + }, +} + +func (m *SignedBlindedBeaconBlockBellatrixV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBlindedBeaconBlockBellatrixV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBlindedBeaconBlockBellatrixV2.Put(m) + } +} +func SignedBlindedBeaconBlockBellatrixV2FromVTPool() *SignedBlindedBeaconBlockBellatrixV2 { + return vtprotoPool_SignedBlindedBeaconBlockBellatrixV2.Get().(*SignedBlindedBeaconBlockBellatrixV2) +} + +var vtprotoPool_SignedBlindedBeaconBlockCapella = sync.Pool{ + New: func() interface{} { + return &SignedBlindedBeaconBlockCapella{} + }, +} + +func (m *SignedBlindedBeaconBlockCapella) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBlindedBeaconBlockCapella) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBlindedBeaconBlockCapella.Put(m) + } +} +func SignedBlindedBeaconBlockCapellaFromVTPool() *SignedBlindedBeaconBlockCapella { + return vtprotoPool_SignedBlindedBeaconBlockCapella.Get().(*SignedBlindedBeaconBlockCapella) +} + +var vtprotoPool_SignedBlindedBeaconBlockCapellaV2 = sync.Pool{ + New: func() interface{} { + return &SignedBlindedBeaconBlockCapellaV2{} + }, +} + +func (m *SignedBlindedBeaconBlockCapellaV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBlindedBeaconBlockCapellaV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBlindedBeaconBlockCapellaV2.Put(m) + } +} +func SignedBlindedBeaconBlockCapellaV2FromVTPool() *SignedBlindedBeaconBlockCapellaV2 { + return vtprotoPool_SignedBlindedBeaconBlockCapellaV2.Get().(*SignedBlindedBeaconBlockCapellaV2) +} + +var vtprotoPool_SignedBeaconBlockAltair = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockAltair{} + }, +} + +func (m *SignedBeaconBlockAltair) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockAltair) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockAltair.Put(m) + } +} +func SignedBeaconBlockAltairFromVTPool() *SignedBeaconBlockAltair { + return vtprotoPool_SignedBeaconBlockAltair.Get().(*SignedBeaconBlockAltair) +} + +var vtprotoPool_SignedBeaconBlockAltairV2 = sync.Pool{ + New: func() interface{} { + return &SignedBeaconBlockAltairV2{} + }, +} + +func (m *SignedBeaconBlockAltairV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBeaconBlockAltairV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBeaconBlockAltairV2.Put(m) + } +} +func SignedBeaconBlockAltairV2FromVTPool() *SignedBeaconBlockAltairV2 { + return vtprotoPool_SignedBeaconBlockAltairV2.Get().(*SignedBeaconBlockAltairV2) +} + +var vtprotoPool_BeaconBlockBellatrix = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBellatrix{} + }, +} + +func (m *BeaconBlockBellatrix) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockBellatrix) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBellatrix.Put(m) + } +} +func BeaconBlockBellatrixFromVTPool() *BeaconBlockBellatrix { + return vtprotoPool_BeaconBlockBellatrix.Get().(*BeaconBlockBellatrix) +} + +var vtprotoPool_BeaconBlockBellatrixV2 = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBellatrixV2{} + }, +} + +func (m *BeaconBlockBellatrixV2) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockBellatrixV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBellatrixV2.Put(m) + } +} +func BeaconBlockBellatrixV2FromVTPool() *BeaconBlockBellatrixV2 { + return vtprotoPool_BeaconBlockBellatrixV2.Get().(*BeaconBlockBellatrixV2) +} + +var vtprotoPool_BlindedBeaconBlockBellatrix = sync.Pool{ + New: func() interface{} { + return &BlindedBeaconBlockBellatrix{} + }, +} + +func (m *BlindedBeaconBlockBellatrix) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BlindedBeaconBlockBellatrix) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlindedBeaconBlockBellatrix.Put(m) + } +} +func BlindedBeaconBlockBellatrixFromVTPool() *BlindedBeaconBlockBellatrix { + return vtprotoPool_BlindedBeaconBlockBellatrix.Get().(*BlindedBeaconBlockBellatrix) +} + +var vtprotoPool_BlindedBeaconBlockBellatrixV2 = sync.Pool{ + New: func() interface{} { + return &BlindedBeaconBlockBellatrixV2{} + }, +} + +func (m *BlindedBeaconBlockBellatrixV2) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BlindedBeaconBlockBellatrixV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlindedBeaconBlockBellatrixV2.Put(m) + } +} +func BlindedBeaconBlockBellatrixV2FromVTPool() *BlindedBeaconBlockBellatrixV2 { + return vtprotoPool_BlindedBeaconBlockBellatrixV2.Get().(*BlindedBeaconBlockBellatrixV2) +} + +var vtprotoPool_BeaconBlockCapella = sync.Pool{ + New: func() interface{} { + return &BeaconBlockCapella{} + }, +} + +func (m *BeaconBlockCapella) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockCapella) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockCapella.Put(m) + } +} +func BeaconBlockCapellaFromVTPool() *BeaconBlockCapella { + return vtprotoPool_BeaconBlockCapella.Get().(*BeaconBlockCapella) +} + +var vtprotoPool_BeaconBlockCapellaV2 = sync.Pool{ + New: func() interface{} { + return &BeaconBlockCapellaV2{} + }, +} + +func (m *BeaconBlockCapellaV2) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockCapellaV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockCapellaV2.Put(m) + } +} +func BeaconBlockCapellaV2FromVTPool() *BeaconBlockCapellaV2 { + return vtprotoPool_BeaconBlockCapellaV2.Get().(*BeaconBlockCapellaV2) +} + +var vtprotoPool_BeaconBlockDeneb = sync.Pool{ + New: func() interface{} { + return &BeaconBlockDeneb{} + }, +} + +func (m *BeaconBlockDeneb) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockDeneb) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockDeneb.Put(m) + } +} +func BeaconBlockDenebFromVTPool() *BeaconBlockDeneb { + return vtprotoPool_BeaconBlockDeneb.Get().(*BeaconBlockDeneb) +} + +var vtprotoPool_BeaconBlockElectra = sync.Pool{ + New: func() interface{} { + return &BeaconBlockElectra{} + }, +} + +func (m *BeaconBlockElectra) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockElectra) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockElectra.Put(m) + } +} +func BeaconBlockElectraFromVTPool() *BeaconBlockElectra { + return vtprotoPool_BeaconBlockElectra.Get().(*BeaconBlockElectra) +} + +var vtprotoPool_BeaconBlockFulu = sync.Pool{ + New: func() interface{} { + return &BeaconBlockFulu{} + }, +} + +func (m *BeaconBlockFulu) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockFulu) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockFulu.Put(m) + } +} +func BeaconBlockFuluFromVTPool() *BeaconBlockFulu { + return vtprotoPool_BeaconBlockFulu.Get().(*BeaconBlockFulu) +} + +var vtprotoPool_BlindedBeaconBlockCapella = sync.Pool{ + New: func() interface{} { + return &BlindedBeaconBlockCapella{} + }, +} + +func (m *BlindedBeaconBlockCapella) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BlindedBeaconBlockCapella) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlindedBeaconBlockCapella.Put(m) + } +} +func BlindedBeaconBlockCapellaFromVTPool() *BlindedBeaconBlockCapella { + return vtprotoPool_BlindedBeaconBlockCapella.Get().(*BlindedBeaconBlockCapella) +} + +var vtprotoPool_BlindedBeaconBlockCapellaV2 = sync.Pool{ + New: func() interface{} { + return &BlindedBeaconBlockCapellaV2{} + }, +} + +func (m *BlindedBeaconBlockCapellaV2) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BlindedBeaconBlockCapellaV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlindedBeaconBlockCapellaV2.Put(m) + } +} +func BlindedBeaconBlockCapellaV2FromVTPool() *BlindedBeaconBlockCapellaV2 { + return vtprotoPool_BlindedBeaconBlockCapellaV2.Get().(*BlindedBeaconBlockCapellaV2) +} + +var vtprotoPool_BeaconBlockAltair = sync.Pool{ + New: func() interface{} { + return &BeaconBlockAltair{} + }, +} + +func (m *BeaconBlockAltair) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockAltair) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockAltair.Put(m) + } +} +func BeaconBlockAltairFromVTPool() *BeaconBlockAltair { + return vtprotoPool_BeaconBlockAltair.Get().(*BeaconBlockAltair) +} + +var vtprotoPool_BeaconBlockAltairV2 = sync.Pool{ + New: func() interface{} { + return &BeaconBlockAltairV2{} + }, +} + +func (m *BeaconBlockAltairV2) ResetVT() { + if m != nil { + m.Body.ReturnToVTPool() + m.Reset() + } +} +func (m *BeaconBlockAltairV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockAltairV2.Put(m) + } +} +func BeaconBlockAltairV2FromVTPool() *BeaconBlockAltairV2 { + return vtprotoPool_BeaconBlockAltairV2.Get().(*BeaconBlockAltairV2) +} + +var vtprotoPool_BeaconBlockBodyBellatrix = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyBellatrix{} + }, +} + +func (m *BeaconBlockBodyBellatrix) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayload.ReturnToVTPool() + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + } +} +func (m *BeaconBlockBodyBellatrix) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyBellatrix.Put(m) + } +} +func BeaconBlockBodyBellatrixFromVTPool() *BeaconBlockBodyBellatrix { + return vtprotoPool_BeaconBlockBodyBellatrix.Get().(*BeaconBlockBodyBellatrix) +} + +var vtprotoPool_BeaconBlockBodyBellatrixV2 = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyBellatrixV2{} + }, +} + +func (m *BeaconBlockBodyBellatrixV2) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayload.ReturnToVTPool() + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + } +} +func (m *BeaconBlockBodyBellatrixV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyBellatrixV2.Put(m) + } +} +func BeaconBlockBodyBellatrixV2FromVTPool() *BeaconBlockBodyBellatrixV2 { + return vtprotoPool_BeaconBlockBodyBellatrixV2.Get().(*BeaconBlockBodyBellatrixV2) +} + +var vtprotoPool_BlindedBeaconBlockBodyBellatrix = sync.Pool{ + New: func() interface{} { + return &BlindedBeaconBlockBodyBellatrix{} + }, +} + +func (m *BlindedBeaconBlockBodyBellatrix) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayloadHeader.ReturnToVTPool() + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + } +} +func (m *BlindedBeaconBlockBodyBellatrix) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlindedBeaconBlockBodyBellatrix.Put(m) + } +} +func BlindedBeaconBlockBodyBellatrixFromVTPool() *BlindedBeaconBlockBodyBellatrix { + return vtprotoPool_BlindedBeaconBlockBodyBellatrix.Get().(*BlindedBeaconBlockBodyBellatrix) +} + +var vtprotoPool_BlindedBeaconBlockBodyBellatrixV2 = sync.Pool{ + New: func() interface{} { + return &BlindedBeaconBlockBodyBellatrixV2{} + }, +} + +func (m *BlindedBeaconBlockBodyBellatrixV2) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayloadHeader.ReturnToVTPool() + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + } +} +func (m *BlindedBeaconBlockBodyBellatrixV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlindedBeaconBlockBodyBellatrixV2.Put(m) + } +} +func BlindedBeaconBlockBodyBellatrixV2FromVTPool() *BlindedBeaconBlockBodyBellatrixV2 { + return vtprotoPool_BlindedBeaconBlockBodyBellatrixV2.Get().(*BlindedBeaconBlockBodyBellatrixV2) +} + +var vtprotoPool_BeaconBlockBodyCapella = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyCapella{} + }, +} + +func (m *BeaconBlockBodyCapella) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayload.ReturnToVTPool() + for _, mm := range m.BlsToExecutionChanges { + mm.ResetVT() + } + f5 := m.BlsToExecutionChanges[:0] + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + m.BlsToExecutionChanges = f5 + } +} +func (m *BeaconBlockBodyCapella) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyCapella.Put(m) + } +} +func BeaconBlockBodyCapellaFromVTPool() *BeaconBlockBodyCapella { + return vtprotoPool_BeaconBlockBodyCapella.Get().(*BeaconBlockBodyCapella) +} + +var vtprotoPool_BeaconBlockBodyCapellaV2 = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyCapellaV2{} + }, +} + +func (m *BeaconBlockBodyCapellaV2) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayload.ReturnToVTPool() + for _, mm := range m.BlsToExecutionChanges { + mm.ResetVT() + } + f5 := m.BlsToExecutionChanges[:0] + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + m.BlsToExecutionChanges = f5 + } +} +func (m *BeaconBlockBodyCapellaV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyCapellaV2.Put(m) + } +} +func BeaconBlockBodyCapellaV2FromVTPool() *BeaconBlockBodyCapellaV2 { + return vtprotoPool_BeaconBlockBodyCapellaV2.Get().(*BeaconBlockBodyCapellaV2) +} + +var vtprotoPool_BeaconBlockBodyDeneb = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyDeneb{} + }, +} + +func (m *BeaconBlockBodyDeneb) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayload.ReturnToVTPool() + for _, mm := range m.BlsToExecutionChanges { + mm.ResetVT() + } + f5 := m.BlsToExecutionChanges[:0] + f6 := m.BlobKzgCommitments[:0] + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + m.BlsToExecutionChanges = f5 + m.BlobKzgCommitments = f6 + } +} +func (m *BeaconBlockBodyDeneb) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyDeneb.Put(m) + } +} +func BeaconBlockBodyDenebFromVTPool() *BeaconBlockBodyDeneb { + return vtprotoPool_BeaconBlockBodyDeneb.Get().(*BeaconBlockBodyDeneb) +} + +var vtprotoPool_BeaconBlockBodyElectra = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyElectra{} + }, +} + +func (m *BeaconBlockBodyElectra) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayload.ReturnToVTPool() + for _, mm := range m.BlsToExecutionChanges { + mm.ResetVT() + } + f5 := m.BlsToExecutionChanges[:0] + f6 := m.BlobKzgCommitments[:0] + m.ExecutionRequests.ReturnToVTPool() + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + m.BlsToExecutionChanges = f5 + m.BlobKzgCommitments = f6 + } +} +func (m *BeaconBlockBodyElectra) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyElectra.Put(m) + } +} +func BeaconBlockBodyElectraFromVTPool() *BeaconBlockBodyElectra { + return vtprotoPool_BeaconBlockBodyElectra.Get().(*BeaconBlockBodyElectra) +} + +var vtprotoPool_BeaconBlockBodyFulu = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyFulu{} + }, +} + +func (m *BeaconBlockBodyFulu) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayload.ReturnToVTPool() + for _, mm := range m.BlsToExecutionChanges { + mm.ResetVT() + } + f5 := m.BlsToExecutionChanges[:0] + f6 := m.BlobKzgCommitments[:0] + m.ExecutionRequests.ReturnToVTPool() + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + m.BlsToExecutionChanges = f5 + m.BlobKzgCommitments = f6 + } +} +func (m *BeaconBlockBodyFulu) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyFulu.Put(m) + } +} +func BeaconBlockBodyFuluFromVTPool() *BeaconBlockBodyFulu { + return vtprotoPool_BeaconBlockBodyFulu.Get().(*BeaconBlockBodyFulu) +} + +var vtprotoPool_BlindedBeaconBlockBodyCapella = sync.Pool{ + New: func() interface{} { + return &BlindedBeaconBlockBodyCapella{} + }, +} + +func (m *BlindedBeaconBlockBodyCapella) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayloadHeader.ReturnToVTPool() + for _, mm := range m.BlsToExecutionChanges { + mm.ResetVT() + } + f5 := m.BlsToExecutionChanges[:0] + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + m.BlsToExecutionChanges = f5 + } +} +func (m *BlindedBeaconBlockBodyCapella) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlindedBeaconBlockBodyCapella.Put(m) + } +} +func BlindedBeaconBlockBodyCapellaFromVTPool() *BlindedBeaconBlockBodyCapella { + return vtprotoPool_BlindedBeaconBlockBodyCapella.Get().(*BlindedBeaconBlockBodyCapella) +} + +var vtprotoPool_BlindedBeaconBlockBodyCapellaV2 = sync.Pool{ + New: func() interface{} { + return &BlindedBeaconBlockBodyCapellaV2{} + }, +} + +func (m *BlindedBeaconBlockBodyCapellaV2) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.ExecutionPayloadHeader.ReturnToVTPool() + for _, mm := range m.BlsToExecutionChanges { + mm.ResetVT() + } + f5 := m.BlsToExecutionChanges[:0] + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + m.BlsToExecutionChanges = f5 + } +} +func (m *BlindedBeaconBlockBodyCapellaV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlindedBeaconBlockBodyCapellaV2.Put(m) + } +} +func BlindedBeaconBlockBodyCapellaV2FromVTPool() *BlindedBeaconBlockBodyCapellaV2 { + return vtprotoPool_BlindedBeaconBlockBodyCapellaV2.Get().(*BlindedBeaconBlockBodyCapellaV2) +} + +var vtprotoPool_BeaconBlockBodyAltair = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyAltair{} + }, +} + +func (m *BeaconBlockBodyAltair) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + } +} +func (m *BeaconBlockBodyAltair) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyAltair.Put(m) + } +} +func BeaconBlockBodyAltairFromVTPool() *BeaconBlockBodyAltair { + return vtprotoPool_BeaconBlockBodyAltair.Get().(*BeaconBlockBodyAltair) +} + +var vtprotoPool_BeaconBlockBodyAltairV2 = sync.Pool{ + New: func() interface{} { + return &BeaconBlockBodyAltairV2{} + }, +} + +func (m *BeaconBlockBodyAltairV2) ResetVT() { + if m != nil { + m.Eth1Data.ReturnToVTPool() + for _, mm := range m.ProposerSlashings { + mm.ResetVT() + } + f0 := m.ProposerSlashings[:0] + for _, mm := range m.AttesterSlashings { + mm.ResetVT() + } + f1 := m.AttesterSlashings[:0] + for _, mm := range m.Attestations { + mm.ResetVT() + } + f2 := m.Attestations[:0] + for _, mm := range m.Deposits { + mm.ResetVT() + } + f3 := m.Deposits[:0] + for _, mm := range m.VoluntaryExits { + mm.ResetVT() + } + f4 := m.VoluntaryExits[:0] + m.SyncAggregate.ReturnToVTPool() + m.Reset() + m.ProposerSlashings = f0 + m.AttesterSlashings = f1 + m.Attestations = f2 + m.Deposits = f3 + m.VoluntaryExits = f4 + } +} +func (m *BeaconBlockBodyAltairV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlockBodyAltairV2.Put(m) + } +} +func BeaconBlockBodyAltairV2FromVTPool() *BeaconBlockBodyAltairV2 { + return vtprotoPool_BeaconBlockBodyAltairV2.Get().(*BeaconBlockBodyAltairV2) +} func (m *SignedBeaconBlockBellatrix) SizeVT() (n int) { if m == nil { return 0 @@ -5027,7 +6277,7 @@ func (m *SignedBeaconBlockBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockBellatrix{} + m.Message = BeaconBlockBellatrixFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5146,7 +6396,7 @@ func (m *SignedBeaconBlockBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockBellatrixV2{} + m.Message = BeaconBlockBellatrixV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5265,7 +6515,7 @@ func (m *SignedBeaconBlockCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockCapella{} + m.Message = BeaconBlockCapellaFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5384,7 +6634,7 @@ func (m *SignedBeaconBlockCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockCapellaV2{} + m.Message = BeaconBlockCapellaV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5503,7 +6753,7 @@ func (m *SignedBeaconBlockDeneb) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockDeneb{} + m.Message = BeaconBlockDenebFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5622,7 +6872,7 @@ func (m *SignedBeaconBlockElectra) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockElectra{} + m.Message = BeaconBlockElectraFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5741,7 +6991,7 @@ func (m *SignedBeaconBlockFulu) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockFulu{} + m.Message = BeaconBlockFuluFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5860,7 +7110,7 @@ func (m *SignedBlindedBeaconBlockBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BlindedBeaconBlockBellatrix{} + m.Message = BlindedBeaconBlockBellatrixFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5979,7 +7229,7 @@ func (m *SignedBlindedBeaconBlockBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BlindedBeaconBlockBellatrixV2{} + m.Message = BlindedBeaconBlockBellatrixV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6098,7 +7348,7 @@ func (m *SignedBlindedBeaconBlockCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BlindedBeaconBlockCapella{} + m.Message = BlindedBeaconBlockCapellaFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6217,7 +7467,7 @@ func (m *SignedBlindedBeaconBlockCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BlindedBeaconBlockCapellaV2{} + m.Message = BlindedBeaconBlockCapellaV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6336,7 +7586,7 @@ func (m *SignedBeaconBlockAltair) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockAltair{} + m.Message = BeaconBlockAltairFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6455,7 +7705,7 @@ func (m *SignedBeaconBlockAltairV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BeaconBlockAltairV2{} + m.Message = BeaconBlockAltairV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6676,7 +7926,7 @@ func (m *BeaconBlockBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyBellatrix{} + m.Body = BeaconBlockBodyBellatrixFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6899,7 +8149,7 @@ func (m *BeaconBlockBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyBellatrixV2{} + m.Body = BeaconBlockBodyBellatrixV2FromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -7088,7 +8338,7 @@ func (m *BlindedBeaconBlockBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BlindedBeaconBlockBodyBellatrix{} + m.Body = BlindedBeaconBlockBodyBellatrixFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -7311,7 +8561,7 @@ func (m *BlindedBeaconBlockBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BlindedBeaconBlockBodyBellatrixV2{} + m.Body = BlindedBeaconBlockBodyBellatrixV2FromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -7500,7 +8750,7 @@ func (m *BeaconBlockCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyCapella{} + m.Body = BeaconBlockBodyCapellaFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -7723,7 +8973,7 @@ func (m *BeaconBlockCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyCapellaV2{} + m.Body = BeaconBlockBodyCapellaV2FromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -7946,7 +9196,7 @@ func (m *BeaconBlockDeneb) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyDeneb{} + m.Body = BeaconBlockBodyDenebFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8169,7 +9419,7 @@ func (m *BeaconBlockElectra) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyElectra{} + m.Body = BeaconBlockBodyElectraFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8392,7 +9642,7 @@ func (m *BeaconBlockFulu) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyFulu{} + m.Body = BeaconBlockBodyFuluFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8581,7 +9831,7 @@ func (m *BlindedBeaconBlockCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BlindedBeaconBlockBodyCapella{} + m.Body = BlindedBeaconBlockBodyCapellaFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8804,7 +10054,7 @@ func (m *BlindedBeaconBlockCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BlindedBeaconBlockBodyCapellaV2{} + m.Body = BlindedBeaconBlockBodyCapellaV2FromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8993,7 +10243,7 @@ func (m *BeaconBlockAltair) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyAltair{} + m.Body = BeaconBlockBodyAltairFromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9216,7 +10466,7 @@ func (m *BeaconBlockAltairV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Body == nil { - m.Body = &BeaconBlockBodyAltairV2{} + m.Body = BeaconBlockBodyAltairV2FromVTPool() } if err := m.Body.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9335,7 +10585,7 @@ func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9402,7 +10652,14 @@ func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9436,7 +10693,14 @@ func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9470,7 +10734,14 @@ func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9504,7 +10775,14 @@ func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9538,7 +10816,14 @@ func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9573,7 +10858,7 @@ func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9609,7 +10894,7 @@ func (m *BeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayload == nil { - m.ExecutionPayload = &v1.ExecutionPayload{} + m.ExecutionPayload = v1.ExecutionPayloadFromVTPool() } if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9728,7 +11013,7 @@ func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9795,7 +11080,14 @@ func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9829,7 +11121,14 @@ func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9863,7 +11162,14 @@ func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9897,7 +11203,14 @@ func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9931,7 +11244,14 @@ func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9966,7 +11286,7 @@ func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10002,7 +11322,7 @@ func (m *BeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayload == nil { - m.ExecutionPayload = &v1.ExecutionPayloadV2{} + m.ExecutionPayload = v1.ExecutionPayloadV2FromVTPool() } if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10121,7 +11441,7 @@ func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10188,7 +11508,14 @@ func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10222,7 +11549,14 @@ func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10256,7 +11590,14 @@ func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10290,7 +11631,14 @@ func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10324,7 +11672,14 @@ func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10359,7 +11714,7 @@ func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10395,7 +11750,7 @@ func (m *BlindedBeaconBlockBodyBellatrix) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayloadHeader == nil { - m.ExecutionPayloadHeader = &v1.ExecutionPayloadHeader{} + m.ExecutionPayloadHeader = v1.ExecutionPayloadHeaderFromVTPool() } if err := m.ExecutionPayloadHeader.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10514,7 +11869,7 @@ func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10581,7 +11936,14 @@ func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10615,7 +11977,14 @@ func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10649,7 +12018,14 @@ func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10683,7 +12059,14 @@ func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10717,7 +12100,14 @@ func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10752,7 +12142,7 @@ func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10788,7 +12178,7 @@ func (m *BlindedBeaconBlockBodyBellatrixV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayloadHeader == nil { - m.ExecutionPayloadHeader = &v1.ExecutionPayloadHeaderV2{} + m.ExecutionPayloadHeader = v1.ExecutionPayloadHeaderV2FromVTPool() } if err := m.ExecutionPayloadHeader.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10907,7 +12297,7 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10974,7 +12364,14 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11008,7 +12405,14 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11042,7 +12446,14 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11076,7 +12487,14 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11110,7 +12528,14 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11145,7 +12570,7 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -11181,7 +12606,7 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayload == nil { - m.ExecutionPayload = &v1.ExecutionPayloadCapella{} + m.ExecutionPayload = v1.ExecutionPayloadCapellaFromVTPool() } if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -11216,7 +12641,14 @@ func (m *BeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if len(m.BlsToExecutionChanges) == cap(m.BlsToExecutionChanges) { + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + } else { + m.BlsToExecutionChanges = m.BlsToExecutionChanges[:len(m.BlsToExecutionChanges)+1] + if m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] == nil { + m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] = &SignedBLSToExecutionChange{} + } + } if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11334,7 +12766,7 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -11401,7 +12833,14 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11435,7 +12874,14 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11469,7 +12915,14 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11503,7 +12956,14 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11537,7 +12997,14 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11572,7 +13039,7 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -11608,7 +13075,7 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayload == nil { - m.ExecutionPayload = &v1.ExecutionPayloadCapellaV2{} + m.ExecutionPayload = v1.ExecutionPayloadCapellaV2FromVTPool() } if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -11643,7 +13110,14 @@ func (m *BeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if len(m.BlsToExecutionChanges) == cap(m.BlsToExecutionChanges) { + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + } else { + m.BlsToExecutionChanges = m.BlsToExecutionChanges[:len(m.BlsToExecutionChanges)+1] + if m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] == nil { + m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] = &SignedBLSToExecutionChange{} + } + } if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11761,7 +13235,7 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -11828,7 +13302,14 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11862,7 +13343,14 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11896,7 +13384,14 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11930,7 +13425,14 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11964,7 +13466,14 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11999,7 +13508,7 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -12035,7 +13544,7 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayload == nil { - m.ExecutionPayload = &v1.ExecutionPayloadDeneb{} + m.ExecutionPayload = v1.ExecutionPayloadDenebFromVTPool() } if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -12070,7 +13579,14 @@ func (m *BeaconBlockBodyDeneb) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if len(m.BlsToExecutionChanges) == cap(m.BlsToExecutionChanges) { + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + } else { + m.BlsToExecutionChanges = m.BlsToExecutionChanges[:len(m.BlsToExecutionChanges)+1] + if m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] == nil { + m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] = &SignedBLSToExecutionChange{} + } + } if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12220,7 +13736,7 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -12287,7 +13803,14 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12321,7 +13844,14 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12355,7 +13885,14 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12389,7 +13926,14 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12423,7 +13967,14 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12458,7 +14009,7 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -12494,7 +14045,7 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayload == nil { - m.ExecutionPayload = &v1.ExecutionPayloadElectra{} + m.ExecutionPayload = v1.ExecutionPayloadElectraFromVTPool() } if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -12529,7 +14080,14 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if len(m.BlsToExecutionChanges) == cap(m.BlsToExecutionChanges) { + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + } else { + m.BlsToExecutionChanges = m.BlsToExecutionChanges[:len(m.BlsToExecutionChanges)+1] + if m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] == nil { + m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] = &SignedBLSToExecutionChange{} + } + } if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12596,7 +14154,7 @@ func (m *BeaconBlockBodyElectra) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionRequests == nil { - m.ExecutionRequests = &v1.ElectraExecutionRequests{} + m.ExecutionRequests = v1.ElectraExecutionRequestsFromVTPool() } if err := m.ExecutionRequests.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -12715,7 +14273,7 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -12782,7 +14340,14 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12816,7 +14381,14 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12850,7 +14422,14 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12884,7 +14463,14 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12918,7 +14504,14 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -12953,7 +14546,7 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -12989,7 +14582,7 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayload == nil { - m.ExecutionPayload = &v1.ExecutionPayloadElectra{} + m.ExecutionPayload = v1.ExecutionPayloadElectraFromVTPool() } if err := m.ExecutionPayload.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -13024,7 +14617,14 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if len(m.BlsToExecutionChanges) == cap(m.BlsToExecutionChanges) { + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + } else { + m.BlsToExecutionChanges = m.BlsToExecutionChanges[:len(m.BlsToExecutionChanges)+1] + if m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] == nil { + m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] = &SignedBLSToExecutionChange{} + } + } if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13091,7 +14691,7 @@ func (m *BeaconBlockBodyFulu) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionRequests == nil { - m.ExecutionRequests = &v1.ElectraExecutionRequests{} + m.ExecutionRequests = v1.ElectraExecutionRequestsFromVTPool() } if err := m.ExecutionRequests.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -13210,7 +14810,7 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -13277,7 +14877,14 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13311,7 +14918,14 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13345,7 +14959,14 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13379,7 +15000,14 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13413,7 +15041,14 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13448,7 +15083,7 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -13484,7 +15119,7 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayloadHeader == nil { - m.ExecutionPayloadHeader = &v1.ExecutionPayloadHeaderCapella{} + m.ExecutionPayloadHeader = v1.ExecutionPayloadHeaderCapellaFromVTPool() } if err := m.ExecutionPayloadHeader.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -13519,7 +15154,14 @@ func (m *BlindedBeaconBlockBodyCapella) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if len(m.BlsToExecutionChanges) == cap(m.BlsToExecutionChanges) { + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + } else { + m.BlsToExecutionChanges = m.BlsToExecutionChanges[:len(m.BlsToExecutionChanges)+1] + if m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] == nil { + m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] = &SignedBLSToExecutionChange{} + } + } if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13637,7 +15279,7 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -13704,7 +15346,14 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13738,7 +15387,14 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13772,7 +15428,14 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13806,7 +15469,14 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13840,7 +15510,14 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -13875,7 +15552,7 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -13911,7 +15588,7 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ExecutionPayloadHeader == nil { - m.ExecutionPayloadHeader = &v1.ExecutionPayloadHeaderCapellaV2{} + m.ExecutionPayloadHeader = v1.ExecutionPayloadHeaderCapellaV2FromVTPool() } if err := m.ExecutionPayloadHeader.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -13946,7 +15623,14 @@ func (m *BlindedBeaconBlockBodyCapellaV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + if len(m.BlsToExecutionChanges) == cap(m.BlsToExecutionChanges) { + m.BlsToExecutionChanges = append(m.BlsToExecutionChanges, &SignedBLSToExecutionChange{}) + } else { + m.BlsToExecutionChanges = m.BlsToExecutionChanges[:len(m.BlsToExecutionChanges)+1] + if m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] == nil { + m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1] = &SignedBLSToExecutionChange{} + } + } if err := m.BlsToExecutionChanges[len(m.BlsToExecutionChanges)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14064,7 +15748,7 @@ func (m *BeaconBlockBodyAltair) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -14131,7 +15815,14 @@ func (m *BeaconBlockBodyAltair) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14165,7 +15856,14 @@ func (m *BeaconBlockBodyAltair) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14199,7 +15897,14 @@ func (m *BeaconBlockBodyAltair) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14233,7 +15938,14 @@ func (m *BeaconBlockBodyAltair) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14267,7 +15979,14 @@ func (m *BeaconBlockBodyAltair) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14302,7 +16021,7 @@ func (m *BeaconBlockBodyAltair) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -14421,7 +16140,7 @@ func (m *BeaconBlockBodyAltairV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Eth1Data == nil { - m.Eth1Data = &v1.Eth1Data{} + m.Eth1Data = v1.Eth1DataFromVTPool() } if err := m.Eth1Data.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -14488,7 +16207,14 @@ func (m *BeaconBlockBodyAltairV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + if len(m.ProposerSlashings) == cap(m.ProposerSlashings) { + m.ProposerSlashings = append(m.ProposerSlashings, &v1.ProposerSlashing{}) + } else { + m.ProposerSlashings = m.ProposerSlashings[:len(m.ProposerSlashings)+1] + if m.ProposerSlashings[len(m.ProposerSlashings)-1] == nil { + m.ProposerSlashings[len(m.ProposerSlashings)-1] = &v1.ProposerSlashing{} + } + } if err := m.ProposerSlashings[len(m.ProposerSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14522,7 +16248,14 @@ func (m *BeaconBlockBodyAltairV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + if len(m.AttesterSlashings) == cap(m.AttesterSlashings) { + m.AttesterSlashings = append(m.AttesterSlashings, &v1.AttesterSlashing{}) + } else { + m.AttesterSlashings = m.AttesterSlashings[:len(m.AttesterSlashings)+1] + if m.AttesterSlashings[len(m.AttesterSlashings)-1] == nil { + m.AttesterSlashings[len(m.AttesterSlashings)-1] = &v1.AttesterSlashing{} + } + } if err := m.AttesterSlashings[len(m.AttesterSlashings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14556,7 +16289,14 @@ func (m *BeaconBlockBodyAltairV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Attestations = append(m.Attestations, &v1.Attestation{}) + if len(m.Attestations) == cap(m.Attestations) { + m.Attestations = append(m.Attestations, &v1.Attestation{}) + } else { + m.Attestations = m.Attestations[:len(m.Attestations)+1] + if m.Attestations[len(m.Attestations)-1] == nil { + m.Attestations[len(m.Attestations)-1] = &v1.Attestation{} + } + } if err := m.Attestations[len(m.Attestations)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14590,7 +16330,14 @@ func (m *BeaconBlockBodyAltairV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Deposits = append(m.Deposits, &v1.Deposit{}) + if len(m.Deposits) == cap(m.Deposits) { + m.Deposits = append(m.Deposits, &v1.Deposit{}) + } else { + m.Deposits = m.Deposits[:len(m.Deposits)+1] + if m.Deposits[len(m.Deposits)-1] == nil { + m.Deposits[len(m.Deposits)-1] = &v1.Deposit{} + } + } if err := m.Deposits[len(m.Deposits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14624,7 +16371,14 @@ func (m *BeaconBlockBodyAltairV2) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + if len(m.VoluntaryExits) == cap(m.VoluntaryExits) { + m.VoluntaryExits = append(m.VoluntaryExits, &v1.SignedVoluntaryExit{}) + } else { + m.VoluntaryExits = m.VoluntaryExits[:len(m.VoluntaryExits)+1] + if m.VoluntaryExits[len(m.VoluntaryExits)-1] == nil { + m.VoluntaryExits[len(m.VoluntaryExits)-1] = &v1.SignedVoluntaryExit{} + } + } if err := m.VoluntaryExits[len(m.VoluntaryExits)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -14659,7 +16413,7 @@ func (m *BeaconBlockBodyAltairV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncAggregate == nil { - m.SyncAggregate = &v1.SyncAggregate{} + m.SyncAggregate = v1.SyncAggregateFromVTPool() } if err := m.SyncAggregate.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err diff --git a/pkg/proto/eth/v2/events_vtproto.pb.go b/pkg/proto/eth/v2/events_vtproto.pb.go index 1bf065bc..a693a63f 100644 --- a/pkg/proto/eth/v2/events_vtproto.pb.go +++ b/pkg/proto/eth/v2/events_vtproto.pb.go @@ -10,6 +10,7 @@ import ( protohelpers "github.com/planetscale/vtprotobuf/protohelpers" protoimpl "google.golang.org/protobuf/runtime/protoimpl" io "io" + sync "sync" ) const ( @@ -393,6 +394,90 @@ func (m *EventBlockV2_FuluBlock) MarshalToSizedBufferVT(dAtA []byte) (int, error } return len(dAtA) - i, nil } + +var vtprotoPool_EventBlock = sync.Pool{ + New: func() interface{} { + return &EventBlock{} + }, +} + +func (m *EventBlock) ResetVT() { + if m != nil { + if oneof, ok := m.Message.(*EventBlock_Phase0Block); ok { + oneof.Phase0Block.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlock_AltairBlock); ok { + oneof.AltairBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlock_BellatrixBlock); ok { + oneof.BellatrixBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlock_CapellaBlock); ok { + oneof.CapellaBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlock_DenebBlock); ok { + oneof.DenebBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlock_ElectraBlock); ok { + oneof.ElectraBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlock_FuluBlock); ok { + oneof.FuluBlock.ReturnToVTPool() + } + m.Reset() + } +} +func (m *EventBlock) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventBlock.Put(m) + } +} +func EventBlockFromVTPool() *EventBlock { + return vtprotoPool_EventBlock.Get().(*EventBlock) +} + +var vtprotoPool_EventBlockV2 = sync.Pool{ + New: func() interface{} { + return &EventBlockV2{} + }, +} + +func (m *EventBlockV2) ResetVT() { + if m != nil { + if oneof, ok := m.Message.(*EventBlockV2_Phase0Block); ok { + oneof.Phase0Block.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlockV2_AltairBlock); ok { + oneof.AltairBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlockV2_BellatrixBlock); ok { + oneof.BellatrixBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlockV2_CapellaBlock); ok { + oneof.CapellaBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlockV2_DenebBlock); ok { + oneof.DenebBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlockV2_ElectraBlock); ok { + oneof.ElectraBlock.ReturnToVTPool() + } + if oneof, ok := m.Message.(*EventBlockV2_FuluBlock); ok { + oneof.FuluBlock.ReturnToVTPool() + } + m.Reset() + } +} +func (m *EventBlockV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EventBlockV2.Put(m) + } +} +func EventBlockV2FromVTPool() *EventBlockV2 { + return vtprotoPool_EventBlockV2.Get().(*EventBlockV2) +} func (m *EventBlock) SizeVT() (n int) { if m == nil { return 0 @@ -664,7 +749,7 @@ func (m *EventBlock) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.BeaconBlock{} + v := v1.BeaconBlockFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -705,7 +790,7 @@ func (m *EventBlock) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockAltair{} + v := BeaconBlockAltairFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -746,7 +831,7 @@ func (m *EventBlock) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockBellatrix{} + v := BeaconBlockBellatrixFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -787,7 +872,7 @@ func (m *EventBlock) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockCapella{} + v := BeaconBlockCapellaFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -879,7 +964,7 @@ func (m *EventBlock) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockDeneb{} + v := BeaconBlockDenebFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -920,7 +1005,7 @@ func (m *EventBlock) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockElectra{} + v := BeaconBlockElectraFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -961,7 +1046,7 @@ func (m *EventBlock) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockFulu{} + v := BeaconBlockFuluFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1053,7 +1138,7 @@ func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.BeaconBlockV2{} + v := v1.BeaconBlockV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1094,7 +1179,7 @@ func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockAltairV2{} + v := BeaconBlockAltairV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1135,7 +1220,7 @@ func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockBellatrixV2{} + v := BeaconBlockBellatrixV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1176,7 +1261,7 @@ func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockCapellaV2{} + v := BeaconBlockCapellaV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1268,7 +1353,7 @@ func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockDeneb{} + v := BeaconBlockDenebFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1309,7 +1394,7 @@ func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockElectra{} + v := BeaconBlockElectraFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1350,7 +1435,7 @@ func (m *EventBlockV2) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &BeaconBlockFulu{} + v := BeaconBlockFuluFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/eth/v2/withdrawals_vtproto.pb.go b/pkg/proto/eth/v2/withdrawals_vtproto.pb.go index 7b125578..1221cb17 100644 --- a/pkg/proto/eth/v2/withdrawals_vtproto.pb.go +++ b/pkg/proto/eth/v2/withdrawals_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -274,6 +275,117 @@ func (m *SubmitBLSToExecutionChangesRequest) MarshalToSizedBufferVT(dAtA []byte) return len(dAtA) - i, nil } +var vtprotoPool_BLSToExecutionChange = sync.Pool{ + New: func() interface{} { + return &BLSToExecutionChange{} + }, +} + +func (m *BLSToExecutionChange) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BLSToExecutionChange) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BLSToExecutionChange.Put(m) + } +} +func BLSToExecutionChangeFromVTPool() *BLSToExecutionChange { + return vtprotoPool_BLSToExecutionChange.Get().(*BLSToExecutionChange) +} + +var vtprotoPool_BLSToExecutionChangeV2 = sync.Pool{ + New: func() interface{} { + return &BLSToExecutionChangeV2{} + }, +} + +func (m *BLSToExecutionChangeV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BLSToExecutionChangeV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BLSToExecutionChangeV2.Put(m) + } +} +func BLSToExecutionChangeV2FromVTPool() *BLSToExecutionChangeV2 { + return vtprotoPool_BLSToExecutionChangeV2.Get().(*BLSToExecutionChangeV2) +} + +var vtprotoPool_SignedBLSToExecutionChange = sync.Pool{ + New: func() interface{} { + return &SignedBLSToExecutionChange{} + }, +} + +func (m *SignedBLSToExecutionChange) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBLSToExecutionChange) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBLSToExecutionChange.Put(m) + } +} +func SignedBLSToExecutionChangeFromVTPool() *SignedBLSToExecutionChange { + return vtprotoPool_SignedBLSToExecutionChange.Get().(*SignedBLSToExecutionChange) +} + +var vtprotoPool_SignedBLSToExecutionChangeV2 = sync.Pool{ + New: func() interface{} { + return &SignedBLSToExecutionChangeV2{} + }, +} + +func (m *SignedBLSToExecutionChangeV2) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *SignedBLSToExecutionChangeV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SignedBLSToExecutionChangeV2.Put(m) + } +} +func SignedBLSToExecutionChangeV2FromVTPool() *SignedBLSToExecutionChangeV2 { + return vtprotoPool_SignedBLSToExecutionChangeV2.Get().(*SignedBLSToExecutionChangeV2) +} + +var vtprotoPool_SubmitBLSToExecutionChangesRequest = sync.Pool{ + New: func() interface{} { + return &SubmitBLSToExecutionChangesRequest{} + }, +} + +func (m *SubmitBLSToExecutionChangesRequest) ResetVT() { + if m != nil { + for _, mm := range m.Changes { + mm.ResetVT() + } + f0 := m.Changes[:0] + m.Reset() + m.Changes = f0 + } +} +func (m *SubmitBLSToExecutionChangesRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SubmitBLSToExecutionChangesRequest.Put(m) + } +} +func SubmitBLSToExecutionChangesRequestFromVTPool() *SubmitBLSToExecutionChangesRequest { + return vtprotoPool_SubmitBLSToExecutionChangesRequest.Get().(*SubmitBLSToExecutionChangesRequest) +} func (m *BLSToExecutionChange) SizeVT() (n int) { if m == nil { return 0 @@ -713,7 +825,7 @@ func (m *SignedBLSToExecutionChange) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BLSToExecutionChange{} + m.Message = BLSToExecutionChangeFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -832,7 +944,7 @@ func (m *SignedBLSToExecutionChangeV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &BLSToExecutionChangeV2{} + m.Message = BLSToExecutionChangeV2FromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -950,7 +1062,14 @@ func (m *SubmitBLSToExecutionChangesRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Changes = append(m.Changes, &SignedBLSToExecutionChange{}) + if len(m.Changes) == cap(m.Changes) { + m.Changes = append(m.Changes, &SignedBLSToExecutionChange{}) + } else { + m.Changes = m.Changes[:len(m.Changes)+1] + if m.Changes[len(m.Changes)-1] == nil { + m.Changes[len(m.Changes)-1] = &SignedBLSToExecutionChange{} + } + } if err := m.Changes[len(m.Changes)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/libp2p/eth_vtproto.pb.go b/pkg/proto/libp2p/eth_vtproto.pb.go index c674474d..af684e07 100644 --- a/pkg/proto/libp2p/eth_vtproto.pb.go +++ b/pkg/proto/libp2p/eth_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -186,6 +187,47 @@ func (m *Metadata) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_Status = sync.Pool{ + New: func() interface{} { + return &Status{} + }, +} + +func (m *Status) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Status) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Status.Put(m) + } +} +func StatusFromVTPool() *Status { + return vtprotoPool_Status.Get().(*Status) +} + +var vtprotoPool_Metadata = sync.Pool{ + New: func() interface{} { + return &Metadata{} + }, +} + +func (m *Metadata) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Metadata) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Metadata.Put(m) + } +} +func MetadataFromVTPool() *Metadata { + return vtprotoPool_Metadata.Get().(*Metadata) +} func (m *Status) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go b/pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go index 36a7f0a1..fbc38543 100644 --- a/pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go +++ b/pkg/proto/libp2p/gossipsub/eth_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -279,6 +280,68 @@ func (m *DataColumnSidecar) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_BeaconBlock = sync.Pool{ + New: func() interface{} { + return &BeaconBlock{} + }, +} + +func (m *BeaconBlock) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BeaconBlock) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BeaconBlock.Put(m) + } +} +func BeaconBlockFromVTPool() *BeaconBlock { + return vtprotoPool_BeaconBlock.Get().(*BeaconBlock) +} + +var vtprotoPool_BlobSidecar = sync.Pool{ + New: func() interface{} { + return &BlobSidecar{} + }, +} + +func (m *BlobSidecar) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BlobSidecar) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlobSidecar.Put(m) + } +} +func BlobSidecarFromVTPool() *BlobSidecar { + return vtprotoPool_BlobSidecar.Get().(*BlobSidecar) +} + +var vtprotoPool_DataColumnSidecar = sync.Pool{ + New: func() interface{} { + return &DataColumnSidecar{} + }, +} + +func (m *DataColumnSidecar) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *DataColumnSidecar) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DataColumnSidecar.Put(m) + } +} +func DataColumnSidecarFromVTPool() *DataColumnSidecar { + return vtprotoPool_DataColumnSidecar.Get().(*DataColumnSidecar) +} func (m *BeaconBlock) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/libp2p/peer_vtproto.pb.go b/pkg/proto/libp2p/peer_vtproto.pb.go index d021498e..d5fa35d3 100644 --- a/pkg/proto/libp2p/peer_vtproto.pb.go +++ b/pkg/proto/libp2p/peer_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -110,6 +111,28 @@ func (m *Peer) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_Peer = sync.Pool{ + New: func() interface{} { + return &Peer{} + }, +} + +func (m *Peer) ResetVT() { + if m != nil { + f0 := m.Protocols[:0] + m.Reset() + m.Protocols = f0 + } +} +func (m *Peer) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Peer.Put(m) + } +} +func PeerFromVTPool() *Peer { + return vtprotoPool_Peer.Get().(*Peer) +} func (m *Peer) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/libp2p/trace_vtproto.pb.go b/pkg/proto/libp2p/trace_vtproto.pb.go index 121d8f87..7c435374 100644 --- a/pkg/proto/libp2p/trace_vtproto.pb.go +++ b/pkg/proto/libp2p/trace_vtproto.pb.go @@ -13,6 +13,7 @@ import ( timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -2834,6 +2835,850 @@ func (m *Identify) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_AddPeer = sync.Pool{ + New: func() interface{} { + return &AddPeer{} + }, +} + +func (m *AddPeer) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *AddPeer) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AddPeer.Put(m) + } +} +func AddPeerFromVTPool() *AddPeer { + return vtprotoPool_AddPeer.Get().(*AddPeer) +} + +var vtprotoPool_RemovePeer = sync.Pool{ + New: func() interface{} { + return &RemovePeer{} + }, +} + +func (m *RemovePeer) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *RemovePeer) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_RemovePeer.Put(m) + } +} +func RemovePeerFromVTPool() *RemovePeer { + return vtprotoPool_RemovePeer.Get().(*RemovePeer) +} + +var vtprotoPool_RecvRPC = sync.Pool{ + New: func() interface{} { + return &RecvRPC{} + }, +} + +func (m *RecvRPC) ResetVT() { + if m != nil { + m.Meta.ReturnToVTPool() + m.Reset() + } +} +func (m *RecvRPC) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_RecvRPC.Put(m) + } +} +func RecvRPCFromVTPool() *RecvRPC { + return vtprotoPool_RecvRPC.Get().(*RecvRPC) +} + +var vtprotoPool_SendRPC = sync.Pool{ + New: func() interface{} { + return &SendRPC{} + }, +} + +func (m *SendRPC) ResetVT() { + if m != nil { + m.Meta.ReturnToVTPool() + m.Reset() + } +} +func (m *SendRPC) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SendRPC.Put(m) + } +} +func SendRPCFromVTPool() *SendRPC { + return vtprotoPool_SendRPC.Get().(*SendRPC) +} + +var vtprotoPool_DropRPC = sync.Pool{ + New: func() interface{} { + return &DropRPC{} + }, +} + +func (m *DropRPC) ResetVT() { + if m != nil { + m.Meta.ReturnToVTPool() + m.Reset() + } +} +func (m *DropRPC) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DropRPC.Put(m) + } +} +func DropRPCFromVTPool() *DropRPC { + return vtprotoPool_DropRPC.Get().(*DropRPC) +} + +var vtprotoPool_Join = sync.Pool{ + New: func() interface{} { + return &Join{} + }, +} + +func (m *Join) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Join) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Join.Put(m) + } +} +func JoinFromVTPool() *Join { + return vtprotoPool_Join.Get().(*Join) +} + +var vtprotoPool_Leave = sync.Pool{ + New: func() interface{} { + return &Leave{} + }, +} + +func (m *Leave) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Leave) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Leave.Put(m) + } +} +func LeaveFromVTPool() *Leave { + return vtprotoPool_Leave.Get().(*Leave) +} + +var vtprotoPool_Graft = sync.Pool{ + New: func() interface{} { + return &Graft{} + }, +} + +func (m *Graft) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Graft) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Graft.Put(m) + } +} +func GraftFromVTPool() *Graft { + return vtprotoPool_Graft.Get().(*Graft) +} + +var vtprotoPool_Prune = sync.Pool{ + New: func() interface{} { + return &Prune{} + }, +} + +func (m *Prune) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Prune) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Prune.Put(m) + } +} +func PruneFromVTPool() *Prune { + return vtprotoPool_Prune.Get().(*Prune) +} + +var vtprotoPool_PublishMessage = sync.Pool{ + New: func() interface{} { + return &PublishMessage{} + }, +} + +func (m *PublishMessage) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *PublishMessage) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_PublishMessage.Put(m) + } +} +func PublishMessageFromVTPool() *PublishMessage { + return vtprotoPool_PublishMessage.Get().(*PublishMessage) +} + +var vtprotoPool_RejectMessage = sync.Pool{ + New: func() interface{} { + return &RejectMessage{} + }, +} + +func (m *RejectMessage) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *RejectMessage) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_RejectMessage.Put(m) + } +} +func RejectMessageFromVTPool() *RejectMessage { + return vtprotoPool_RejectMessage.Get().(*RejectMessage) +} + +var vtprotoPool_DuplicateMessage = sync.Pool{ + New: func() interface{} { + return &DuplicateMessage{} + }, +} + +func (m *DuplicateMessage) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *DuplicateMessage) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DuplicateMessage.Put(m) + } +} +func DuplicateMessageFromVTPool() *DuplicateMessage { + return vtprotoPool_DuplicateMessage.Get().(*DuplicateMessage) +} + +var vtprotoPool_DeliverMessage = sync.Pool{ + New: func() interface{} { + return &DeliverMessage{} + }, +} + +func (m *DeliverMessage) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *DeliverMessage) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DeliverMessage.Put(m) + } +} +func DeliverMessageFromVTPool() *DeliverMessage { + return vtprotoPool_DeliverMessage.Get().(*DeliverMessage) +} + +var vtprotoPool_TraceEventMetadata = sync.Pool{ + New: func() interface{} { + return &TraceEventMetadata{} + }, +} + +func (m *TraceEventMetadata) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *TraceEventMetadata) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_TraceEventMetadata.Put(m) + } +} +func TraceEventMetadataFromVTPool() *TraceEventMetadata { + return vtprotoPool_TraceEventMetadata.Get().(*TraceEventMetadata) +} + +var vtprotoPool_Connected = sync.Pool{ + New: func() interface{} { + return &Connected{} + }, +} + +func (m *Connected) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Connected) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Connected.Put(m) + } +} +func ConnectedFromVTPool() *Connected { + return vtprotoPool_Connected.Get().(*Connected) +} + +var vtprotoPool_Disconnected = sync.Pool{ + New: func() interface{} { + return &Disconnected{} + }, +} + +func (m *Disconnected) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Disconnected) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Disconnected.Put(m) + } +} +func DisconnectedFromVTPool() *Disconnected { + return vtprotoPool_Disconnected.Get().(*Disconnected) +} + +var vtprotoPool_HandleStatus = sync.Pool{ + New: func() interface{} { + return &HandleStatus{} + }, +} + +func (m *HandleStatus) ResetVT() { + if m != nil { + m.Request.ReturnToVTPool() + m.Response.ReturnToVTPool() + m.Reset() + } +} +func (m *HandleStatus) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_HandleStatus.Put(m) + } +} +func HandleStatusFromVTPool() *HandleStatus { + return vtprotoPool_HandleStatus.Get().(*HandleStatus) +} + +var vtprotoPool_HandleMetadata = sync.Pool{ + New: func() interface{} { + return &HandleMetadata{} + }, +} + +func (m *HandleMetadata) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *HandleMetadata) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_HandleMetadata.Put(m) + } +} +func HandleMetadataFromVTPool() *HandleMetadata { + return vtprotoPool_HandleMetadata.Get().(*HandleMetadata) +} + +var vtprotoPool_SyntheticHeartbeat = sync.Pool{ + New: func() interface{} { + return &SyntheticHeartbeat{} + }, +} + +func (m *SyntheticHeartbeat) ResetVT() { + if m != nil { + f0 := m.Protocols[:0] + m.Reset() + m.Protocols = f0 + } +} +func (m *SyntheticHeartbeat) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SyntheticHeartbeat.Put(m) + } +} +func SyntheticHeartbeatFromVTPool() *SyntheticHeartbeat { + return vtprotoPool_SyntheticHeartbeat.Get().(*SyntheticHeartbeat) +} + +var vtprotoPool_RPCMeta = sync.Pool{ + New: func() interface{} { + return &RPCMeta{} + }, +} + +func (m *RPCMeta) ResetVT() { + if m != nil { + for _, mm := range m.Messages { + mm.ResetVT() + } + f0 := m.Messages[:0] + for _, mm := range m.Subscriptions { + mm.ResetVT() + } + f1 := m.Subscriptions[:0] + m.Control.ReturnToVTPool() + m.Reset() + m.Messages = f0 + m.Subscriptions = f1 + } +} +func (m *RPCMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_RPCMeta.Put(m) + } +} +func RPCMetaFromVTPool() *RPCMeta { + return vtprotoPool_RPCMeta.Get().(*RPCMeta) +} + +var vtprotoPool_MessageMeta = sync.Pool{ + New: func() interface{} { + return &MessageMeta{} + }, +} + +func (m *MessageMeta) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *MessageMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_MessageMeta.Put(m) + } +} +func MessageMetaFromVTPool() *MessageMeta { + return vtprotoPool_MessageMeta.Get().(*MessageMeta) +} + +var vtprotoPool_MessageMetaItem = sync.Pool{ + New: func() interface{} { + return &MessageMetaItem{} + }, +} + +func (m *MessageMetaItem) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *MessageMetaItem) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_MessageMetaItem.Put(m) + } +} +func MessageMetaItemFromVTPool() *MessageMetaItem { + return vtprotoPool_MessageMetaItem.Get().(*MessageMetaItem) +} + +var vtprotoPool_SubMeta = sync.Pool{ + New: func() interface{} { + return &SubMeta{} + }, +} + +func (m *SubMeta) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *SubMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SubMeta.Put(m) + } +} +func SubMetaFromVTPool() *SubMeta { + return vtprotoPool_SubMeta.Get().(*SubMeta) +} + +var vtprotoPool_SubMetaItem = sync.Pool{ + New: func() interface{} { + return &SubMetaItem{} + }, +} + +func (m *SubMetaItem) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *SubMetaItem) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SubMetaItem.Put(m) + } +} +func SubMetaItemFromVTPool() *SubMetaItem { + return vtprotoPool_SubMetaItem.Get().(*SubMetaItem) +} + +var vtprotoPool_ControlMeta = sync.Pool{ + New: func() interface{} { + return &ControlMeta{} + }, +} + +func (m *ControlMeta) ResetVT() { + if m != nil { + for _, mm := range m.Ihave { + mm.ResetVT() + } + f0 := m.Ihave[:0] + for _, mm := range m.Iwant { + mm.ResetVT() + } + f1 := m.Iwant[:0] + for _, mm := range m.Graft { + mm.ResetVT() + } + f2 := m.Graft[:0] + for _, mm := range m.Prune { + mm.ResetVT() + } + f3 := m.Prune[:0] + for _, mm := range m.Idontwant { + mm.ResetVT() + } + f4 := m.Idontwant[:0] + m.Reset() + m.Ihave = f0 + m.Iwant = f1 + m.Graft = f2 + m.Prune = f3 + m.Idontwant = f4 + } +} +func (m *ControlMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlMeta.Put(m) + } +} +func ControlMetaFromVTPool() *ControlMeta { + return vtprotoPool_ControlMeta.Get().(*ControlMeta) +} + +var vtprotoPool_ControlIHaveMeta = sync.Pool{ + New: func() interface{} { + return &ControlIHaveMeta{} + }, +} + +func (m *ControlIHaveMeta) ResetVT() { + if m != nil { + for _, mm := range m.MessageIds { + mm.Reset() + } + f0 := m.MessageIds[:0] + m.Reset() + m.MessageIds = f0 + } +} +func (m *ControlIHaveMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlIHaveMeta.Put(m) + } +} +func ControlIHaveMetaFromVTPool() *ControlIHaveMeta { + return vtprotoPool_ControlIHaveMeta.Get().(*ControlIHaveMeta) +} + +var vtprotoPool_ControlIHaveMetaItem = sync.Pool{ + New: func() interface{} { + return &ControlIHaveMetaItem{} + }, +} + +func (m *ControlIHaveMetaItem) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ControlIHaveMetaItem) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlIHaveMetaItem.Put(m) + } +} +func ControlIHaveMetaItemFromVTPool() *ControlIHaveMetaItem { + return vtprotoPool_ControlIHaveMetaItem.Get().(*ControlIHaveMetaItem) +} + +var vtprotoPool_ControlIWantMeta = sync.Pool{ + New: func() interface{} { + return &ControlIWantMeta{} + }, +} + +func (m *ControlIWantMeta) ResetVT() { + if m != nil { + for _, mm := range m.MessageIds { + mm.Reset() + } + f0 := m.MessageIds[:0] + m.Reset() + m.MessageIds = f0 + } +} +func (m *ControlIWantMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlIWantMeta.Put(m) + } +} +func ControlIWantMetaFromVTPool() *ControlIWantMeta { + return vtprotoPool_ControlIWantMeta.Get().(*ControlIWantMeta) +} + +var vtprotoPool_ControlIWantMetaItem = sync.Pool{ + New: func() interface{} { + return &ControlIWantMetaItem{} + }, +} + +func (m *ControlIWantMetaItem) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ControlIWantMetaItem) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlIWantMetaItem.Put(m) + } +} +func ControlIWantMetaItemFromVTPool() *ControlIWantMetaItem { + return vtprotoPool_ControlIWantMetaItem.Get().(*ControlIWantMetaItem) +} + +var vtprotoPool_ControlGraftMeta = sync.Pool{ + New: func() interface{} { + return &ControlGraftMeta{} + }, +} + +func (m *ControlGraftMeta) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ControlGraftMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlGraftMeta.Put(m) + } +} +func ControlGraftMetaFromVTPool() *ControlGraftMeta { + return vtprotoPool_ControlGraftMeta.Get().(*ControlGraftMeta) +} + +var vtprotoPool_ControlGraftMetaItem = sync.Pool{ + New: func() interface{} { + return &ControlGraftMetaItem{} + }, +} + +func (m *ControlGraftMetaItem) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ControlGraftMetaItem) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlGraftMetaItem.Put(m) + } +} +func ControlGraftMetaItemFromVTPool() *ControlGraftMetaItem { + return vtprotoPool_ControlGraftMetaItem.Get().(*ControlGraftMetaItem) +} + +var vtprotoPool_ControlPruneMeta = sync.Pool{ + New: func() interface{} { + return &ControlPruneMeta{} + }, +} + +func (m *ControlPruneMeta) ResetVT() { + if m != nil { + for _, mm := range m.PeerIds { + mm.Reset() + } + f0 := m.PeerIds[:0] + m.Reset() + m.PeerIds = f0 + } +} +func (m *ControlPruneMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlPruneMeta.Put(m) + } +} +func ControlPruneMetaFromVTPool() *ControlPruneMeta { + return vtprotoPool_ControlPruneMeta.Get().(*ControlPruneMeta) +} + +var vtprotoPool_ControlPruneMetaItem = sync.Pool{ + New: func() interface{} { + return &ControlPruneMetaItem{} + }, +} + +func (m *ControlPruneMetaItem) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ControlPruneMetaItem) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlPruneMetaItem.Put(m) + } +} +func ControlPruneMetaItemFromVTPool() *ControlPruneMetaItem { + return vtprotoPool_ControlPruneMetaItem.Get().(*ControlPruneMetaItem) +} + +var vtprotoPool_ControlIDontWantMeta = sync.Pool{ + New: func() interface{} { + return &ControlIDontWantMeta{} + }, +} + +func (m *ControlIDontWantMeta) ResetVT() { + if m != nil { + for _, mm := range m.MessageIds { + mm.Reset() + } + f0 := m.MessageIds[:0] + m.Reset() + m.MessageIds = f0 + } +} +func (m *ControlIDontWantMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlIDontWantMeta.Put(m) + } +} +func ControlIDontWantMetaFromVTPool() *ControlIDontWantMeta { + return vtprotoPool_ControlIDontWantMeta.Get().(*ControlIDontWantMeta) +} + +var vtprotoPool_ControlIDontWantMetaItem = sync.Pool{ + New: func() interface{} { + return &ControlIDontWantMetaItem{} + }, +} + +func (m *ControlIDontWantMetaItem) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ControlIDontWantMetaItem) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ControlIDontWantMetaItem.Put(m) + } +} +func ControlIDontWantMetaItemFromVTPool() *ControlIDontWantMetaItem { + return vtprotoPool_ControlIDontWantMetaItem.Get().(*ControlIDontWantMetaItem) +} + +var vtprotoPool_DataColumnCustodyProbe = sync.Pool{ + New: func() interface{} { + return &DataColumnCustodyProbe{} + }, +} + +func (m *DataColumnCustodyProbe) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *DataColumnCustodyProbe) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DataColumnCustodyProbe.Put(m) + } +} +func DataColumnCustodyProbeFromVTPool() *DataColumnCustodyProbe { + return vtprotoPool_DataColumnCustodyProbe.Get().(*DataColumnCustodyProbe) +} + +var vtprotoPool_Identify = sync.Pool{ + New: func() interface{} { + return &Identify{} + }, +} + +func (m *Identify) ResetVT() { + if m != nil { + f0 := m.Protocols[:0] + f1 := m.ListenAddrs[:0] + m.Reset() + m.Protocols = f0 + m.ListenAddrs = f1 + } +} +func (m *Identify) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Identify.Put(m) + } +} +func IdentifyFromVTPool() *Identify { + return vtprotoPool_Identify.Get().(*Identify) +} func (m *AddPeer) SizeVT() (n int) { if m == nil { return 0 @@ -4165,7 +5010,7 @@ func (m *RecvRPC) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Meta == nil { - m.Meta = &RPCMeta{} + m.Meta = RPCMetaFromVTPool() } if err := m.Meta.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4288,7 +5133,7 @@ func (m *SendRPC) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Meta == nil { - m.Meta = &RPCMeta{} + m.Meta = RPCMetaFromVTPool() } if err := m.Meta.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4411,7 +5256,7 @@ func (m *DropRPC) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Meta == nil { - m.Meta = &RPCMeta{} + m.Meta = RPCMetaFromVTPool() } if err := m.Meta.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6715,7 +7560,7 @@ func (m *HandleStatus) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Request == nil { - m.Request = &Status{} + m.Request = StatusFromVTPool() } if err := m.Request.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6751,7 +7596,7 @@ func (m *HandleStatus) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Response == nil { - m.Response = &Status{} + m.Response = StatusFromVTPool() } if err := m.Response.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -7018,7 +7863,7 @@ func (m *HandleMetadata) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &Metadata{} + m.Metadata = MetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -7475,7 +8320,14 @@ func (m *RPCMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Messages = append(m.Messages, &MessageMeta{}) + if len(m.Messages) == cap(m.Messages) { + m.Messages = append(m.Messages, &MessageMeta{}) + } else { + m.Messages = m.Messages[:len(m.Messages)+1] + if m.Messages[len(m.Messages)-1] == nil { + m.Messages[len(m.Messages)-1] = &MessageMeta{} + } + } if err := m.Messages[len(m.Messages)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -7509,7 +8361,14 @@ func (m *RPCMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Subscriptions = append(m.Subscriptions, &SubMeta{}) + if len(m.Subscriptions) == cap(m.Subscriptions) { + m.Subscriptions = append(m.Subscriptions, &SubMeta{}) + } else { + m.Subscriptions = m.Subscriptions[:len(m.Subscriptions)+1] + if m.Subscriptions[len(m.Subscriptions)-1] == nil { + m.Subscriptions[len(m.Subscriptions)-1] = &SubMeta{} + } + } if err := m.Subscriptions[len(m.Subscriptions)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -7544,7 +8403,7 @@ func (m *RPCMeta) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Control == nil { - m.Control = &ControlMeta{} + m.Control = ControlMetaFromVTPool() } if err := m.Control.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8374,7 +9233,14 @@ func (m *ControlMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Ihave = append(m.Ihave, &ControlIHaveMeta{}) + if len(m.Ihave) == cap(m.Ihave) { + m.Ihave = append(m.Ihave, &ControlIHaveMeta{}) + } else { + m.Ihave = m.Ihave[:len(m.Ihave)+1] + if m.Ihave[len(m.Ihave)-1] == nil { + m.Ihave[len(m.Ihave)-1] = &ControlIHaveMeta{} + } + } if err := m.Ihave[len(m.Ihave)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8408,7 +9274,14 @@ func (m *ControlMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Iwant = append(m.Iwant, &ControlIWantMeta{}) + if len(m.Iwant) == cap(m.Iwant) { + m.Iwant = append(m.Iwant, &ControlIWantMeta{}) + } else { + m.Iwant = m.Iwant[:len(m.Iwant)+1] + if m.Iwant[len(m.Iwant)-1] == nil { + m.Iwant[len(m.Iwant)-1] = &ControlIWantMeta{} + } + } if err := m.Iwant[len(m.Iwant)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8442,7 +9315,14 @@ func (m *ControlMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Graft = append(m.Graft, &ControlGraftMeta{}) + if len(m.Graft) == cap(m.Graft) { + m.Graft = append(m.Graft, &ControlGraftMeta{}) + } else { + m.Graft = m.Graft[:len(m.Graft)+1] + if m.Graft[len(m.Graft)-1] == nil { + m.Graft[len(m.Graft)-1] = &ControlGraftMeta{} + } + } if err := m.Graft[len(m.Graft)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8476,7 +9356,14 @@ func (m *ControlMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Prune = append(m.Prune, &ControlPruneMeta{}) + if len(m.Prune) == cap(m.Prune) { + m.Prune = append(m.Prune, &ControlPruneMeta{}) + } else { + m.Prune = m.Prune[:len(m.Prune)+1] + if m.Prune[len(m.Prune)-1] == nil { + m.Prune[len(m.Prune)-1] = &ControlPruneMeta{} + } + } if err := m.Prune[len(m.Prune)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8510,7 +9397,14 @@ func (m *ControlMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Idontwant = append(m.Idontwant, &ControlIDontWantMeta{}) + if len(m.Idontwant) == cap(m.Idontwant) { + m.Idontwant = append(m.Idontwant, &ControlIDontWantMeta{}) + } else { + m.Idontwant = m.Idontwant[:len(m.Idontwant)+1] + if m.Idontwant[len(m.Idontwant)-1] == nil { + m.Idontwant[len(m.Idontwant)-1] = &ControlIDontWantMeta{} + } + } if err := m.Idontwant[len(m.Idontwant)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8631,7 +9525,14 @@ func (m *ControlIHaveMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + if len(m.MessageIds) == cap(m.MessageIds) { + m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + } else { + m.MessageIds = m.MessageIds[:len(m.MessageIds)+1] + if m.MessageIds[len(m.MessageIds)-1] == nil { + m.MessageIds[len(m.MessageIds)-1] = &wrapperspb1.StringValue{} + } + } if err := (*wrapperspb.StringValue)(m.MessageIds[len(m.MessageIds)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8983,7 +9884,14 @@ func (m *ControlIWantMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + if len(m.MessageIds) == cap(m.MessageIds) { + m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + } else { + m.MessageIds = m.MessageIds[:len(m.MessageIds)+1] + if m.MessageIds[len(m.MessageIds)-1] == nil { + m.MessageIds[len(m.MessageIds)-1] = &wrapperspb1.StringValue{} + } + } if err := (*wrapperspb.StringValue)(m.MessageIds[len(m.MessageIds)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9617,7 +10525,14 @@ func (m *ControlPruneMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.PeerIds = append(m.PeerIds, &wrapperspb1.StringValue{}) + if len(m.PeerIds) == cap(m.PeerIds) { + m.PeerIds = append(m.PeerIds, &wrapperspb1.StringValue{}) + } else { + m.PeerIds = m.PeerIds[:len(m.PeerIds)+1] + if m.PeerIds[len(m.PeerIds)-1] == nil { + m.PeerIds[len(m.PeerIds)-1] = &wrapperspb1.StringValue{} + } + } if err := (*wrapperspb.StringValue)(m.PeerIds[len(m.PeerIds)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9969,7 +10884,14 @@ func (m *ControlIDontWantMeta) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + if len(m.MessageIds) == cap(m.MessageIds) { + m.MessageIds = append(m.MessageIds, &wrapperspb1.StringValue{}) + } else { + m.MessageIds = m.MessageIds[:len(m.MessageIds)+1] + if m.MessageIds[len(m.MessageIds)-1] == nil { + m.MessageIds[len(m.MessageIds)-1] = &wrapperspb1.StringValue{} + } + } if err := (*wrapperspb.StringValue)(m.MessageIds[len(m.MessageIds)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/proto/mevrelay/bids_vtproto.pb.go b/pkg/proto/mevrelay/bids_vtproto.pb.go index d8bc60d2..df886928 100644 --- a/pkg/proto/mevrelay/bids_vtproto.pb.go +++ b/pkg/proto/mevrelay/bids_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -193,6 +194,26 @@ func (m *BidTrace) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_BidTrace = sync.Pool{ + New: func() interface{} { + return &BidTrace{} + }, +} + +func (m *BidTrace) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BidTrace) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BidTrace.Put(m) + } +} +func BidTraceFromVTPool() *BidTrace { + return vtprotoPool_BidTrace.Get().(*BidTrace) +} func (m *BidTrace) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/mevrelay/payloads_vtproto.pb.go b/pkg/proto/mevrelay/payloads_vtproto.pb.go index 9db1d5d6..559dc94e 100644 --- a/pkg/proto/mevrelay/payloads_vtproto.pb.go +++ b/pkg/proto/mevrelay/payloads_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -163,6 +164,26 @@ func (m *ProposerPayloadDelivered) MarshalToSizedBufferVT(dAtA []byte) (int, err return len(dAtA) - i, nil } +var vtprotoPool_ProposerPayloadDelivered = sync.Pool{ + New: func() interface{} { + return &ProposerPayloadDelivered{} + }, +} + +func (m *ProposerPayloadDelivered) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ProposerPayloadDelivered) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ProposerPayloadDelivered.Put(m) + } +} +func ProposerPayloadDeliveredFromVTPool() *ProposerPayloadDelivered { + return vtprotoPool_ProposerPayloadDelivered.Get().(*ProposerPayloadDelivered) +} func (m *ProposerPayloadDelivered) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/mevrelay/relay_vtproto.pb.go b/pkg/proto/mevrelay/relay_vtproto.pb.go index c2e7b4f3..715bd436 100644 --- a/pkg/proto/mevrelay/relay_vtproto.pb.go +++ b/pkg/proto/mevrelay/relay_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -199,6 +200,69 @@ func (m *ValidatorRegistration) MarshalToSizedBufferVT(dAtA []byte) (int, error) return len(dAtA) - i, nil } +var vtprotoPool_Relay = sync.Pool{ + New: func() interface{} { + return &Relay{} + }, +} + +func (m *Relay) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Relay) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Relay.Put(m) + } +} +func RelayFromVTPool() *Relay { + return vtprotoPool_Relay.Get().(*Relay) +} + +var vtprotoPool_ValidatorRegistrationMessage = sync.Pool{ + New: func() interface{} { + return &ValidatorRegistrationMessage{} + }, +} + +func (m *ValidatorRegistrationMessage) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ValidatorRegistrationMessage) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ValidatorRegistrationMessage.Put(m) + } +} +func ValidatorRegistrationMessageFromVTPool() *ValidatorRegistrationMessage { + return vtprotoPool_ValidatorRegistrationMessage.Get().(*ValidatorRegistrationMessage) +} + +var vtprotoPool_ValidatorRegistration = sync.Pool{ + New: func() interface{} { + return &ValidatorRegistration{} + }, +} + +func (m *ValidatorRegistration) ResetVT() { + if m != nil { + m.Message.ReturnToVTPool() + m.Reset() + } +} +func (m *ValidatorRegistration) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ValidatorRegistration.Put(m) + } +} +func ValidatorRegistrationFromVTPool() *ValidatorRegistration { + return vtprotoPool_ValidatorRegistration.Get().(*ValidatorRegistration) +} func (m *Relay) SizeVT() (n int) { if m == nil { return 0 @@ -638,7 +702,7 @@ func (m *ValidatorRegistration) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Message == nil { - m.Message = &ValidatorRegistrationMessage{} + m.Message = ValidatorRegistrationMessageFromVTPool() } if err := m.Message.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err diff --git a/pkg/proto/noderecord/consensus_vtproto.pb.go b/pkg/proto/noderecord/consensus_vtproto.pb.go index 35ce3b6a..b0d8374b 100644 --- a/pkg/proto/noderecord/consensus_vtproto.pb.go +++ b/pkg/proto/noderecord/consensus_vtproto.pb.go @@ -13,6 +13,7 @@ import ( timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -253,6 +254,26 @@ func (m *Consensus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_Consensus = sync.Pool{ + New: func() interface{} { + return &Consensus{} + }, +} + +func (m *Consensus) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Consensus) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Consensus.Put(m) + } +} +func ConsensusFromVTPool() *Consensus { + return vtprotoPool_Consensus.Get().(*Consensus) +} func (m *Consensus) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/noderecord/execution_vtproto.pb.go b/pkg/proto/noderecord/execution_vtproto.pb.go index 598681c6..cdb14143 100644 --- a/pkg/proto/noderecord/execution_vtproto.pb.go +++ b/pkg/proto/noderecord/execution_vtproto.pb.go @@ -13,6 +13,7 @@ import ( timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" + sync "sync" ) const ( @@ -205,6 +206,26 @@ func (m *Execution) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +var vtprotoPool_Execution = sync.Pool{ + New: func() interface{} { + return &Execution{} + }, +} + +func (m *Execution) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Execution) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Execution.Put(m) + } +} +func ExecutionFromVTPool() *Execution { + return vtprotoPool_Execution.Get().(*Execution) +} func (m *Execution) SizeVT() (n int) { if m == nil { return 0 diff --git a/pkg/proto/xatu/coordinator_vtproto.pb.go b/pkg/proto/xatu/coordinator_vtproto.pb.go index 003462ba..1a47521d 100644 --- a/pkg/proto/xatu/coordinator_vtproto.pb.go +++ b/pkg/proto/xatu/coordinator_vtproto.pb.go @@ -11,6 +11,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" timestamppb1 "google.golang.org/protobuf/types/known/timestamppb" io "io" + sync "sync" ) const ( @@ -3060,6 +3061,1330 @@ func (m *UpsertRelayMonitorLocationResponse) MarshalToSizedBufferVT(dAtA []byte) return len(dAtA) - i, nil } +var vtprotoPool_CreateNodeRecordsRequest = sync.Pool{ + New: func() interface{} { + return &CreateNodeRecordsRequest{} + }, +} + +func (m *CreateNodeRecordsRequest) ResetVT() { + if m != nil { + f0 := m.NodeRecords[:0] + m.Reset() + m.NodeRecords = f0 + } +} +func (m *CreateNodeRecordsRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateNodeRecordsRequest.Put(m) + } +} +func CreateNodeRecordsRequestFromVTPool() *CreateNodeRecordsRequest { + return vtprotoPool_CreateNodeRecordsRequest.Get().(*CreateNodeRecordsRequest) +} + +var vtprotoPool_CreateNodeRecordsResponse = sync.Pool{ + New: func() interface{} { + return &CreateNodeRecordsResponse{} + }, +} + +func (m *CreateNodeRecordsResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *CreateNodeRecordsResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateNodeRecordsResponse.Put(m) + } +} +func CreateNodeRecordsResponseFromVTPool() *CreateNodeRecordsResponse { + return vtprotoPool_CreateNodeRecordsResponse.Get().(*CreateNodeRecordsResponse) +} + +var vtprotoPool_ListStalledExecutionNodeRecordsRequest = sync.Pool{ + New: func() interface{} { + return &ListStalledExecutionNodeRecordsRequest{} + }, +} + +func (m *ListStalledExecutionNodeRecordsRequest) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ListStalledExecutionNodeRecordsRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ListStalledExecutionNodeRecordsRequest.Put(m) + } +} +func ListStalledExecutionNodeRecordsRequestFromVTPool() *ListStalledExecutionNodeRecordsRequest { + return vtprotoPool_ListStalledExecutionNodeRecordsRequest.Get().(*ListStalledExecutionNodeRecordsRequest) +} + +var vtprotoPool_ListStalledExecutionNodeRecordsResponse = sync.Pool{ + New: func() interface{} { + return &ListStalledExecutionNodeRecordsResponse{} + }, +} + +func (m *ListStalledExecutionNodeRecordsResponse) ResetVT() { + if m != nil { + f0 := m.NodeRecords[:0] + m.Reset() + m.NodeRecords = f0 + } +} +func (m *ListStalledExecutionNodeRecordsResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ListStalledExecutionNodeRecordsResponse.Put(m) + } +} +func ListStalledExecutionNodeRecordsResponseFromVTPool() *ListStalledExecutionNodeRecordsResponse { + return vtprotoPool_ListStalledExecutionNodeRecordsResponse.Get().(*ListStalledExecutionNodeRecordsResponse) +} + +var vtprotoPool_ExecutionNodeStatus_Capability = sync.Pool{ + New: func() interface{} { + return &ExecutionNodeStatus_Capability{} + }, +} + +func (m *ExecutionNodeStatus_Capability) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionNodeStatus_Capability) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionNodeStatus_Capability.Put(m) + } +} +func ExecutionNodeStatus_CapabilityFromVTPool() *ExecutionNodeStatus_Capability { + return vtprotoPool_ExecutionNodeStatus_Capability.Get().(*ExecutionNodeStatus_Capability) +} + +var vtprotoPool_ExecutionNodeStatus_ForkID = sync.Pool{ + New: func() interface{} { + return &ExecutionNodeStatus_ForkID{} + }, +} + +func (m *ExecutionNodeStatus_ForkID) ResetVT() { + if m != nil { + f0 := m.Hash[:0] + m.Reset() + m.Hash = f0 + } +} +func (m *ExecutionNodeStatus_ForkID) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionNodeStatus_ForkID.Put(m) + } +} +func ExecutionNodeStatus_ForkIDFromVTPool() *ExecutionNodeStatus_ForkID { + return vtprotoPool_ExecutionNodeStatus_ForkID.Get().(*ExecutionNodeStatus_ForkID) +} + +var vtprotoPool_ExecutionNodeStatus = sync.Pool{ + New: func() interface{} { + return &ExecutionNodeStatus{} + }, +} + +func (m *ExecutionNodeStatus) ResetVT() { + if m != nil { + for _, mm := range m.Capabilities { + mm.ResetVT() + } + f0 := m.Capabilities[:0] + f1 := m.Head[:0] + f2 := m.Genesis[:0] + m.ForkId.ReturnToVTPool() + m.Reset() + m.Capabilities = f0 + m.Head = f1 + m.Genesis = f2 + } +} +func (m *ExecutionNodeStatus) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionNodeStatus.Put(m) + } +} +func ExecutionNodeStatusFromVTPool() *ExecutionNodeStatus { + return vtprotoPool_ExecutionNodeStatus.Get().(*ExecutionNodeStatus) +} + +var vtprotoPool_CreateExecutionNodeRecordStatusRequest = sync.Pool{ + New: func() interface{} { + return &CreateExecutionNodeRecordStatusRequest{} + }, +} + +func (m *CreateExecutionNodeRecordStatusRequest) ResetVT() { + if m != nil { + m.Status.ReturnToVTPool() + m.Reset() + } +} +func (m *CreateExecutionNodeRecordStatusRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateExecutionNodeRecordStatusRequest.Put(m) + } +} +func CreateExecutionNodeRecordStatusRequestFromVTPool() *CreateExecutionNodeRecordStatusRequest { + return vtprotoPool_CreateExecutionNodeRecordStatusRequest.Get().(*CreateExecutionNodeRecordStatusRequest) +} + +var vtprotoPool_CreateExecutionNodeRecordStatusResponse = sync.Pool{ + New: func() interface{} { + return &CreateExecutionNodeRecordStatusResponse{} + }, +} + +func (m *CreateExecutionNodeRecordStatusResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *CreateExecutionNodeRecordStatusResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateExecutionNodeRecordStatusResponse.Put(m) + } +} +func CreateExecutionNodeRecordStatusResponseFromVTPool() *CreateExecutionNodeRecordStatusResponse { + return vtprotoPool_CreateExecutionNodeRecordStatusResponse.Get().(*CreateExecutionNodeRecordStatusResponse) +} + +var vtprotoPool_CoordinatedNodeRecord = sync.Pool{ + New: func() interface{} { + return &CoordinatedNodeRecord{} + }, +} + +func (m *CoordinatedNodeRecord) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *CoordinatedNodeRecord) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CoordinatedNodeRecord.Put(m) + } +} +func CoordinatedNodeRecordFromVTPool() *CoordinatedNodeRecord { + return vtprotoPool_CoordinatedNodeRecord.Get().(*CoordinatedNodeRecord) +} + +var vtprotoPool_CoordinateExecutionNodeRecordsRequest = sync.Pool{ + New: func() interface{} { + return &CoordinateExecutionNodeRecordsRequest{} + }, +} + +func (m *CoordinateExecutionNodeRecordsRequest) ResetVT() { + if m != nil { + for _, mm := range m.NodeRecords { + mm.ResetVT() + } + f0 := m.NodeRecords[:0] + f1 := m.NetworkIds[:0] + f2 := m.ForkIdHashes[:0] + f3 := m.Capabilities[:0] + m.Reset() + m.NodeRecords = f0 + m.NetworkIds = f1 + m.ForkIdHashes = f2 + m.Capabilities = f3 + } +} +func (m *CoordinateExecutionNodeRecordsRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CoordinateExecutionNodeRecordsRequest.Put(m) + } +} +func CoordinateExecutionNodeRecordsRequestFromVTPool() *CoordinateExecutionNodeRecordsRequest { + return vtprotoPool_CoordinateExecutionNodeRecordsRequest.Get().(*CoordinateExecutionNodeRecordsRequest) +} + +var vtprotoPool_CoordinateExecutionNodeRecordsResponse = sync.Pool{ + New: func() interface{} { + return &CoordinateExecutionNodeRecordsResponse{} + }, +} + +func (m *CoordinateExecutionNodeRecordsResponse) ResetVT() { + if m != nil { + f0 := m.NodeRecords[:0] + m.Reset() + m.NodeRecords = f0 + } +} +func (m *CoordinateExecutionNodeRecordsResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CoordinateExecutionNodeRecordsResponse.Put(m) + } +} +func CoordinateExecutionNodeRecordsResponseFromVTPool() *CoordinateExecutionNodeRecordsResponse { + return vtprotoPool_CoordinateExecutionNodeRecordsResponse.Get().(*CoordinateExecutionNodeRecordsResponse) +} + +var vtprotoPool_ConsensusNodeStatus = sync.Pool{ + New: func() interface{} { + return &ConsensusNodeStatus{} + }, +} + +func (m *ConsensusNodeStatus) ResetVT() { + if m != nil { + f0 := m.ForkDigest[:0] + f1 := m.NextForkDigest[:0] + f2 := m.FinalizedRoot[:0] + f3 := m.FinalizedEpoch[:0] + f4 := m.HeadRoot[:0] + f5 := m.HeadSlot[:0] + f6 := m.Cgc[:0] + m.Reset() + m.ForkDigest = f0 + m.NextForkDigest = f1 + m.FinalizedRoot = f2 + m.FinalizedEpoch = f3 + m.HeadRoot = f4 + m.HeadSlot = f5 + m.Cgc = f6 + } +} +func (m *ConsensusNodeStatus) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ConsensusNodeStatus.Put(m) + } +} +func ConsensusNodeStatusFromVTPool() *ConsensusNodeStatus { + return vtprotoPool_ConsensusNodeStatus.Get().(*ConsensusNodeStatus) +} + +var vtprotoPool_ListStalledConsensusNodeRecordsRequest = sync.Pool{ + New: func() interface{} { + return &ListStalledConsensusNodeRecordsRequest{} + }, +} + +func (m *ListStalledConsensusNodeRecordsRequest) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ListStalledConsensusNodeRecordsRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ListStalledConsensusNodeRecordsRequest.Put(m) + } +} +func ListStalledConsensusNodeRecordsRequestFromVTPool() *ListStalledConsensusNodeRecordsRequest { + return vtprotoPool_ListStalledConsensusNodeRecordsRequest.Get().(*ListStalledConsensusNodeRecordsRequest) +} + +var vtprotoPool_ListStalledConsensusNodeRecordsResponse = sync.Pool{ + New: func() interface{} { + return &ListStalledConsensusNodeRecordsResponse{} + }, +} + +func (m *ListStalledConsensusNodeRecordsResponse) ResetVT() { + if m != nil { + f0 := m.NodeRecords[:0] + m.Reset() + m.NodeRecords = f0 + } +} +func (m *ListStalledConsensusNodeRecordsResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ListStalledConsensusNodeRecordsResponse.Put(m) + } +} +func ListStalledConsensusNodeRecordsResponseFromVTPool() *ListStalledConsensusNodeRecordsResponse { + return vtprotoPool_ListStalledConsensusNodeRecordsResponse.Get().(*ListStalledConsensusNodeRecordsResponse) +} + +var vtprotoPool_CreateConsensusNodeRecordStatusRequest = sync.Pool{ + New: func() interface{} { + return &CreateConsensusNodeRecordStatusRequest{} + }, +} + +func (m *CreateConsensusNodeRecordStatusRequest) ResetVT() { + if m != nil { + m.Status.ReturnToVTPool() + m.Reset() + } +} +func (m *CreateConsensusNodeRecordStatusRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateConsensusNodeRecordStatusRequest.Put(m) + } +} +func CreateConsensusNodeRecordStatusRequestFromVTPool() *CreateConsensusNodeRecordStatusRequest { + return vtprotoPool_CreateConsensusNodeRecordStatusRequest.Get().(*CreateConsensusNodeRecordStatusRequest) +} + +var vtprotoPool_CreateConsensusNodeRecordStatusResponse = sync.Pool{ + New: func() interface{} { + return &CreateConsensusNodeRecordStatusResponse{} + }, +} + +func (m *CreateConsensusNodeRecordStatusResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *CreateConsensusNodeRecordStatusResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateConsensusNodeRecordStatusResponse.Put(m) + } +} +func CreateConsensusNodeRecordStatusResponseFromVTPool() *CreateConsensusNodeRecordStatusResponse { + return vtprotoPool_CreateConsensusNodeRecordStatusResponse.Get().(*CreateConsensusNodeRecordStatusResponse) +} + +var vtprotoPool_CreateConsensusNodeRecordStatusesRequest = sync.Pool{ + New: func() interface{} { + return &CreateConsensusNodeRecordStatusesRequest{} + }, +} + +func (m *CreateConsensusNodeRecordStatusesRequest) ResetVT() { + if m != nil { + for _, mm := range m.Statuses { + mm.ResetVT() + } + f0 := m.Statuses[:0] + m.Reset() + m.Statuses = f0 + } +} +func (m *CreateConsensusNodeRecordStatusesRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateConsensusNodeRecordStatusesRequest.Put(m) + } +} +func CreateConsensusNodeRecordStatusesRequestFromVTPool() *CreateConsensusNodeRecordStatusesRequest { + return vtprotoPool_CreateConsensusNodeRecordStatusesRequest.Get().(*CreateConsensusNodeRecordStatusesRequest) +} + +var vtprotoPool_CreateConsensusNodeRecordStatusesResponse = sync.Pool{ + New: func() interface{} { + return &CreateConsensusNodeRecordStatusesResponse{} + }, +} + +func (m *CreateConsensusNodeRecordStatusesResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *CreateConsensusNodeRecordStatusesResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateConsensusNodeRecordStatusesResponse.Put(m) + } +} +func CreateConsensusNodeRecordStatusesResponseFromVTPool() *CreateConsensusNodeRecordStatusesResponse { + return vtprotoPool_CreateConsensusNodeRecordStatusesResponse.Get().(*CreateConsensusNodeRecordStatusesResponse) +} + +var vtprotoPool_CoordinateConsensusNodeRecordsRequest = sync.Pool{ + New: func() interface{} { + return &CoordinateConsensusNodeRecordsRequest{} + }, +} + +func (m *CoordinateConsensusNodeRecordsRequest) ResetVT() { + if m != nil { + for _, mm := range m.NodeRecords { + mm.ResetVT() + } + f0 := m.NodeRecords[:0] + f1 := m.NetworkIds[:0] + f2 := m.ForkIdHashes[:0] + f3 := m.Capabilities[:0] + m.Reset() + m.NodeRecords = f0 + m.NetworkIds = f1 + m.ForkIdHashes = f2 + m.Capabilities = f3 + } +} +func (m *CoordinateConsensusNodeRecordsRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CoordinateConsensusNodeRecordsRequest.Put(m) + } +} +func CoordinateConsensusNodeRecordsRequestFromVTPool() *CoordinateConsensusNodeRecordsRequest { + return vtprotoPool_CoordinateConsensusNodeRecordsRequest.Get().(*CoordinateConsensusNodeRecordsRequest) +} + +var vtprotoPool_CoordinateConsensusNodeRecordsResponse = sync.Pool{ + New: func() interface{} { + return &CoordinateConsensusNodeRecordsResponse{} + }, +} + +func (m *CoordinateConsensusNodeRecordsResponse) ResetVT() { + if m != nil { + f0 := m.NodeRecords[:0] + m.Reset() + m.NodeRecords = f0 + } +} +func (m *CoordinateConsensusNodeRecordsResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CoordinateConsensusNodeRecordsResponse.Put(m) + } +} +func CoordinateConsensusNodeRecordsResponseFromVTPool() *CoordinateConsensusNodeRecordsResponse { + return vtprotoPool_CoordinateConsensusNodeRecordsResponse.Get().(*CoordinateConsensusNodeRecordsResponse) +} + +var vtprotoPool_GetDiscoveryNodeRecordRequest = sync.Pool{ + New: func() interface{} { + return &GetDiscoveryNodeRecordRequest{} + }, +} + +func (m *GetDiscoveryNodeRecordRequest) ResetVT() { + if m != nil { + f0 := m.NetworkIds[:0] + f1 := m.ForkIdHashes[:0] + m.Reset() + m.NetworkIds = f0 + m.ForkIdHashes = f1 + } +} +func (m *GetDiscoveryNodeRecordRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetDiscoveryNodeRecordRequest.Put(m) + } +} +func GetDiscoveryNodeRecordRequestFromVTPool() *GetDiscoveryNodeRecordRequest { + return vtprotoPool_GetDiscoveryNodeRecordRequest.Get().(*GetDiscoveryNodeRecordRequest) +} + +var vtprotoPool_GetDiscoveryNodeRecordResponse = sync.Pool{ + New: func() interface{} { + return &GetDiscoveryNodeRecordResponse{} + }, +} + +func (m *GetDiscoveryNodeRecordResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *GetDiscoveryNodeRecordResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetDiscoveryNodeRecordResponse.Put(m) + } +} +func GetDiscoveryNodeRecordResponseFromVTPool() *GetDiscoveryNodeRecordResponse { + return vtprotoPool_GetDiscoveryNodeRecordResponse.Get().(*GetDiscoveryNodeRecordResponse) +} + +var vtprotoPool_GetDiscoveryExecutionNodeRecordRequest = sync.Pool{ + New: func() interface{} { + return &GetDiscoveryExecutionNodeRecordRequest{} + }, +} + +func (m *GetDiscoveryExecutionNodeRecordRequest) ResetVT() { + if m != nil { + f0 := m.NetworkIds[:0] + f1 := m.ForkIdHashes[:0] + m.Reset() + m.NetworkIds = f0 + m.ForkIdHashes = f1 + } +} +func (m *GetDiscoveryExecutionNodeRecordRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetDiscoveryExecutionNodeRecordRequest.Put(m) + } +} +func GetDiscoveryExecutionNodeRecordRequestFromVTPool() *GetDiscoveryExecutionNodeRecordRequest { + return vtprotoPool_GetDiscoveryExecutionNodeRecordRequest.Get().(*GetDiscoveryExecutionNodeRecordRequest) +} + +var vtprotoPool_GetDiscoveryExecutionNodeRecordResponse = sync.Pool{ + New: func() interface{} { + return &GetDiscoveryExecutionNodeRecordResponse{} + }, +} + +func (m *GetDiscoveryExecutionNodeRecordResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *GetDiscoveryExecutionNodeRecordResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetDiscoveryExecutionNodeRecordResponse.Put(m) + } +} +func GetDiscoveryExecutionNodeRecordResponseFromVTPool() *GetDiscoveryExecutionNodeRecordResponse { + return vtprotoPool_GetDiscoveryExecutionNodeRecordResponse.Get().(*GetDiscoveryExecutionNodeRecordResponse) +} + +var vtprotoPool_GetDiscoveryConsensusNodeRecordRequest = sync.Pool{ + New: func() interface{} { + return &GetDiscoveryConsensusNodeRecordRequest{} + }, +} + +func (m *GetDiscoveryConsensusNodeRecordRequest) ResetVT() { + if m != nil { + f0 := m.NetworkIds[:0] + f1 := m.ForkDigests[:0] + m.Reset() + m.NetworkIds = f0 + m.ForkDigests = f1 + } +} +func (m *GetDiscoveryConsensusNodeRecordRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetDiscoveryConsensusNodeRecordRequest.Put(m) + } +} +func GetDiscoveryConsensusNodeRecordRequestFromVTPool() *GetDiscoveryConsensusNodeRecordRequest { + return vtprotoPool_GetDiscoveryConsensusNodeRecordRequest.Get().(*GetDiscoveryConsensusNodeRecordRequest) +} + +var vtprotoPool_GetDiscoveryConsensusNodeRecordResponse = sync.Pool{ + New: func() interface{} { + return &GetDiscoveryConsensusNodeRecordResponse{} + }, +} + +func (m *GetDiscoveryConsensusNodeRecordResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *GetDiscoveryConsensusNodeRecordResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetDiscoveryConsensusNodeRecordResponse.Put(m) + } +} +func GetDiscoveryConsensusNodeRecordResponseFromVTPool() *GetDiscoveryConsensusNodeRecordResponse { + return vtprotoPool_GetDiscoveryConsensusNodeRecordResponse.Get().(*GetDiscoveryConsensusNodeRecordResponse) +} + +var vtprotoPool_BackfillingCheckpointMarker = sync.Pool{ + New: func() interface{} { + return &BackfillingCheckpointMarker{} + }, +} + +func (m *BackfillingCheckpointMarker) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *BackfillingCheckpointMarker) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BackfillingCheckpointMarker.Put(m) + } +} +func BackfillingCheckpointMarkerFromVTPool() *BackfillingCheckpointMarker { + return vtprotoPool_BackfillingCheckpointMarker.Get().(*BackfillingCheckpointMarker) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockVoluntaryExit = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockVoluntaryExit{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockVoluntaryExit) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockVoluntaryExit) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockVoluntaryExit.Put(m) + } +} +func CannonLocationEthV2BeaconBlockVoluntaryExitFromVTPool() *CannonLocationEthV2BeaconBlockVoluntaryExit { + return vtprotoPool_CannonLocationEthV2BeaconBlockVoluntaryExit.Get().(*CannonLocationEthV2BeaconBlockVoluntaryExit) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockProposerSlashing = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockProposerSlashing{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockProposerSlashing) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockProposerSlashing) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockProposerSlashing.Put(m) + } +} +func CannonLocationEthV2BeaconBlockProposerSlashingFromVTPool() *CannonLocationEthV2BeaconBlockProposerSlashing { + return vtprotoPool_CannonLocationEthV2BeaconBlockProposerSlashing.Get().(*CannonLocationEthV2BeaconBlockProposerSlashing) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockDeposit = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockDeposit{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockDeposit) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockDeposit) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockDeposit.Put(m) + } +} +func CannonLocationEthV2BeaconBlockDepositFromVTPool() *CannonLocationEthV2BeaconBlockDeposit { + return vtprotoPool_CannonLocationEthV2BeaconBlockDeposit.Get().(*CannonLocationEthV2BeaconBlockDeposit) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockAttesterSlashing = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockAttesterSlashing{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockAttesterSlashing) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockAttesterSlashing) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockAttesterSlashing.Put(m) + } +} +func CannonLocationEthV2BeaconBlockAttesterSlashingFromVTPool() *CannonLocationEthV2BeaconBlockAttesterSlashing { + return vtprotoPool_CannonLocationEthV2BeaconBlockAttesterSlashing.Get().(*CannonLocationEthV2BeaconBlockAttesterSlashing) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockBlsToExecutionChange = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockBlsToExecutionChange{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockBlsToExecutionChange) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockBlsToExecutionChange) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockBlsToExecutionChange.Put(m) + } +} +func CannonLocationEthV2BeaconBlockBlsToExecutionChangeFromVTPool() *CannonLocationEthV2BeaconBlockBlsToExecutionChange { + return vtprotoPool_CannonLocationEthV2BeaconBlockBlsToExecutionChange.Get().(*CannonLocationEthV2BeaconBlockBlsToExecutionChange) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockExecutionTransaction = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockExecutionTransaction{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockExecutionTransaction) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockExecutionTransaction) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockExecutionTransaction.Put(m) + } +} +func CannonLocationEthV2BeaconBlockExecutionTransactionFromVTPool() *CannonLocationEthV2BeaconBlockExecutionTransaction { + return vtprotoPool_CannonLocationEthV2BeaconBlockExecutionTransaction.Get().(*CannonLocationEthV2BeaconBlockExecutionTransaction) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockWithdrawal = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockWithdrawal{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockWithdrawal) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockWithdrawal) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockWithdrawal.Put(m) + } +} +func CannonLocationEthV2BeaconBlockWithdrawalFromVTPool() *CannonLocationEthV2BeaconBlockWithdrawal { + return vtprotoPool_CannonLocationEthV2BeaconBlockWithdrawal.Get().(*CannonLocationEthV2BeaconBlockWithdrawal) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlock = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlock{} + }, +} + +func (m *CannonLocationEthV2BeaconBlock) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlock) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlock.Put(m) + } +} +func CannonLocationEthV2BeaconBlockFromVTPool() *CannonLocationEthV2BeaconBlock { + return vtprotoPool_CannonLocationEthV2BeaconBlock.Get().(*CannonLocationEthV2BeaconBlock) +} + +var vtprotoPool_CannonLocationEthV1BeaconBlobSidecar = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV1BeaconBlobSidecar{} + }, +} + +func (m *CannonLocationEthV1BeaconBlobSidecar) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV1BeaconBlobSidecar) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV1BeaconBlobSidecar.Put(m) + } +} +func CannonLocationEthV1BeaconBlobSidecarFromVTPool() *CannonLocationEthV1BeaconBlobSidecar { + return vtprotoPool_CannonLocationEthV1BeaconBlobSidecar.Get().(*CannonLocationEthV1BeaconBlobSidecar) +} + +var vtprotoPool_CannonLocationEthV1BeaconProposerDuty = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV1BeaconProposerDuty{} + }, +} + +func (m *CannonLocationEthV1BeaconProposerDuty) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV1BeaconProposerDuty) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV1BeaconProposerDuty.Put(m) + } +} +func CannonLocationEthV1BeaconProposerDutyFromVTPool() *CannonLocationEthV1BeaconProposerDuty { + return vtprotoPool_CannonLocationEthV1BeaconProposerDuty.Get().(*CannonLocationEthV1BeaconProposerDuty) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockElaboratedAttestation = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockElaboratedAttestation{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockElaboratedAttestation) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockElaboratedAttestation) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockElaboratedAttestation.Put(m) + } +} +func CannonLocationEthV2BeaconBlockElaboratedAttestationFromVTPool() *CannonLocationEthV2BeaconBlockElaboratedAttestation { + return vtprotoPool_CannonLocationEthV2BeaconBlockElaboratedAttestation.Get().(*CannonLocationEthV2BeaconBlockElaboratedAttestation) +} + +var vtprotoPool_CannonLocationEthV1BeaconValidators = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV1BeaconValidators{} + }, +} + +func (m *CannonLocationEthV1BeaconValidators) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV1BeaconValidators) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV1BeaconValidators.Put(m) + } +} +func CannonLocationEthV1BeaconValidatorsFromVTPool() *CannonLocationEthV1BeaconValidators { + return vtprotoPool_CannonLocationEthV1BeaconValidators.Get().(*CannonLocationEthV1BeaconValidators) +} + +var vtprotoPool_CannonLocationEthV1BeaconCommittee = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV1BeaconCommittee{} + }, +} + +func (m *CannonLocationEthV1BeaconCommittee) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV1BeaconCommittee) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV1BeaconCommittee.Put(m) + } +} +func CannonLocationEthV1BeaconCommitteeFromVTPool() *CannonLocationEthV1BeaconCommittee { + return vtprotoPool_CannonLocationEthV1BeaconCommittee.Get().(*CannonLocationEthV1BeaconCommittee) +} + +var vtprotoPool_CannonLocationEthV1BeaconSyncCommittee = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV1BeaconSyncCommittee{} + }, +} + +func (m *CannonLocationEthV1BeaconSyncCommittee) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV1BeaconSyncCommittee) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV1BeaconSyncCommittee.Put(m) + } +} +func CannonLocationEthV1BeaconSyncCommitteeFromVTPool() *CannonLocationEthV1BeaconSyncCommittee { + return vtprotoPool_CannonLocationEthV1BeaconSyncCommittee.Get().(*CannonLocationEthV1BeaconSyncCommittee) +} + +var vtprotoPool_CannonLocationEthV2BeaconBlockSyncAggregate = sync.Pool{ + New: func() interface{} { + return &CannonLocationEthV2BeaconBlockSyncAggregate{} + }, +} + +func (m *CannonLocationEthV2BeaconBlockSyncAggregate) ResetVT() { + if m != nil { + m.BackfillingCheckpointMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *CannonLocationEthV2BeaconBlockSyncAggregate) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocationEthV2BeaconBlockSyncAggregate.Put(m) + } +} +func CannonLocationEthV2BeaconBlockSyncAggregateFromVTPool() *CannonLocationEthV2BeaconBlockSyncAggregate { + return vtprotoPool_CannonLocationEthV2BeaconBlockSyncAggregate.Get().(*CannonLocationEthV2BeaconBlockSyncAggregate) +} + +var vtprotoPool_CannonLocation = sync.Pool{ + New: func() interface{} { + return &CannonLocation{} + }, +} + +func (m *CannonLocation) ResetVT() { + if m != nil { + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockVoluntaryExit); ok { + oneof.EthV2BeaconBlockVoluntaryExit.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockProposerSlashing); ok { + oneof.EthV2BeaconBlockProposerSlashing.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockDeposit); ok { + oneof.EthV2BeaconBlockDeposit.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockAttesterSlashing); ok { + oneof.EthV2BeaconBlockAttesterSlashing.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockBlsToExecutionChange); ok { + oneof.EthV2BeaconBlockBlsToExecutionChange.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockExecutionTransaction); ok { + oneof.EthV2BeaconBlockExecutionTransaction.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockWithdrawal); ok { + oneof.EthV2BeaconBlockWithdrawal.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlock); ok { + oneof.EthV2BeaconBlock.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconBlobSidecar); ok { + oneof.EthV1BeaconBlobSidecar.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconProposerDuty); ok { + oneof.EthV1BeaconProposerDuty.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockElaboratedAttestation); ok { + oneof.EthV2BeaconBlockElaboratedAttestation.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconValidators); ok { + oneof.EthV1BeaconValidators.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconCommittee); ok { + oneof.EthV1BeaconCommittee.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV1BeaconSyncCommittee); ok { + oneof.EthV1BeaconSyncCommittee.ReturnToVTPool() + } + if oneof, ok := m.Data.(*CannonLocation_EthV2BeaconBlockSyncAggregate); ok { + oneof.EthV2BeaconBlockSyncAggregate.ReturnToVTPool() + } + m.Reset() + } +} +func (m *CannonLocation) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CannonLocation.Put(m) + } +} +func CannonLocationFromVTPool() *CannonLocation { + return vtprotoPool_CannonLocation.Get().(*CannonLocation) +} + +var vtprotoPool_GetCannonLocationRequest = sync.Pool{ + New: func() interface{} { + return &GetCannonLocationRequest{} + }, +} + +func (m *GetCannonLocationRequest) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *GetCannonLocationRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetCannonLocationRequest.Put(m) + } +} +func GetCannonLocationRequestFromVTPool() *GetCannonLocationRequest { + return vtprotoPool_GetCannonLocationRequest.Get().(*GetCannonLocationRequest) +} + +var vtprotoPool_GetCannonLocationResponse = sync.Pool{ + New: func() interface{} { + return &GetCannonLocationResponse{} + }, +} + +func (m *GetCannonLocationResponse) ResetVT() { + if m != nil { + m.Location.ReturnToVTPool() + m.Reset() + } +} +func (m *GetCannonLocationResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetCannonLocationResponse.Put(m) + } +} +func GetCannonLocationResponseFromVTPool() *GetCannonLocationResponse { + return vtprotoPool_GetCannonLocationResponse.Get().(*GetCannonLocationResponse) +} + +var vtprotoPool_UpsertCannonLocationRequest = sync.Pool{ + New: func() interface{} { + return &UpsertCannonLocationRequest{} + }, +} + +func (m *UpsertCannonLocationRequest) ResetVT() { + if m != nil { + m.Location.ReturnToVTPool() + m.Reset() + } +} +func (m *UpsertCannonLocationRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_UpsertCannonLocationRequest.Put(m) + } +} +func UpsertCannonLocationRequestFromVTPool() *UpsertCannonLocationRequest { + return vtprotoPool_UpsertCannonLocationRequest.Get().(*UpsertCannonLocationRequest) +} + +var vtprotoPool_UpsertCannonLocationResponse = sync.Pool{ + New: func() interface{} { + return &UpsertCannonLocationResponse{} + }, +} + +func (m *UpsertCannonLocationResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *UpsertCannonLocationResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_UpsertCannonLocationResponse.Put(m) + } +} +func UpsertCannonLocationResponseFromVTPool() *UpsertCannonLocationResponse { + return vtprotoPool_UpsertCannonLocationResponse.Get().(*UpsertCannonLocationResponse) +} + +var vtprotoPool_RelayMonitorSlotMarker = sync.Pool{ + New: func() interface{} { + return &RelayMonitorSlotMarker{} + }, +} + +func (m *RelayMonitorSlotMarker) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *RelayMonitorSlotMarker) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_RelayMonitorSlotMarker.Put(m) + } +} +func RelayMonitorSlotMarkerFromVTPool() *RelayMonitorSlotMarker { + return vtprotoPool_RelayMonitorSlotMarker.Get().(*RelayMonitorSlotMarker) +} + +var vtprotoPool_RelayMonitorLocationBidTrace = sync.Pool{ + New: func() interface{} { + return &RelayMonitorLocationBidTrace{} + }, +} + +func (m *RelayMonitorLocationBidTrace) ResetVT() { + if m != nil { + m.SlotMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *RelayMonitorLocationBidTrace) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_RelayMonitorLocationBidTrace.Put(m) + } +} +func RelayMonitorLocationBidTraceFromVTPool() *RelayMonitorLocationBidTrace { + return vtprotoPool_RelayMonitorLocationBidTrace.Get().(*RelayMonitorLocationBidTrace) +} + +var vtprotoPool_RelayMonitorLocationPayloadDelivered = sync.Pool{ + New: func() interface{} { + return &RelayMonitorLocationPayloadDelivered{} + }, +} + +func (m *RelayMonitorLocationPayloadDelivered) ResetVT() { + if m != nil { + m.SlotMarker.ReturnToVTPool() + m.Reset() + } +} +func (m *RelayMonitorLocationPayloadDelivered) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_RelayMonitorLocationPayloadDelivered.Put(m) + } +} +func RelayMonitorLocationPayloadDeliveredFromVTPool() *RelayMonitorLocationPayloadDelivered { + return vtprotoPool_RelayMonitorLocationPayloadDelivered.Get().(*RelayMonitorLocationPayloadDelivered) +} + +var vtprotoPool_RelayMonitorLocation = sync.Pool{ + New: func() interface{} { + return &RelayMonitorLocation{} + }, +} + +func (m *RelayMonitorLocation) ResetVT() { + if m != nil { + if oneof, ok := m.Data.(*RelayMonitorLocation_BidTrace); ok { + oneof.BidTrace.ReturnToVTPool() + } + if oneof, ok := m.Data.(*RelayMonitorLocation_PayloadDelivered); ok { + oneof.PayloadDelivered.ReturnToVTPool() + } + m.Reset() + } +} +func (m *RelayMonitorLocation) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_RelayMonitorLocation.Put(m) + } +} +func RelayMonitorLocationFromVTPool() *RelayMonitorLocation { + return vtprotoPool_RelayMonitorLocation.Get().(*RelayMonitorLocation) +} + +var vtprotoPool_GetRelayMonitorLocationRequest = sync.Pool{ + New: func() interface{} { + return &GetRelayMonitorLocationRequest{} + }, +} + +func (m *GetRelayMonitorLocationRequest) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *GetRelayMonitorLocationRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetRelayMonitorLocationRequest.Put(m) + } +} +func GetRelayMonitorLocationRequestFromVTPool() *GetRelayMonitorLocationRequest { + return vtprotoPool_GetRelayMonitorLocationRequest.Get().(*GetRelayMonitorLocationRequest) +} + +var vtprotoPool_GetRelayMonitorLocationResponse = sync.Pool{ + New: func() interface{} { + return &GetRelayMonitorLocationResponse{} + }, +} + +func (m *GetRelayMonitorLocationResponse) ResetVT() { + if m != nil { + m.Location.ReturnToVTPool() + m.Reset() + } +} +func (m *GetRelayMonitorLocationResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_GetRelayMonitorLocationResponse.Put(m) + } +} +func GetRelayMonitorLocationResponseFromVTPool() *GetRelayMonitorLocationResponse { + return vtprotoPool_GetRelayMonitorLocationResponse.Get().(*GetRelayMonitorLocationResponse) +} + +var vtprotoPool_UpsertRelayMonitorLocationRequest = sync.Pool{ + New: func() interface{} { + return &UpsertRelayMonitorLocationRequest{} + }, +} + +func (m *UpsertRelayMonitorLocationRequest) ResetVT() { + if m != nil { + m.Location.ReturnToVTPool() + m.Reset() + } +} +func (m *UpsertRelayMonitorLocationRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_UpsertRelayMonitorLocationRequest.Put(m) + } +} +func UpsertRelayMonitorLocationRequestFromVTPool() *UpsertRelayMonitorLocationRequest { + return vtprotoPool_UpsertRelayMonitorLocationRequest.Get().(*UpsertRelayMonitorLocationRequest) +} + +var vtprotoPool_UpsertRelayMonitorLocationResponse = sync.Pool{ + New: func() interface{} { + return &UpsertRelayMonitorLocationResponse{} + }, +} + +func (m *UpsertRelayMonitorLocationResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *UpsertRelayMonitorLocationResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_UpsertRelayMonitorLocationResponse.Put(m) + } +} +func UpsertRelayMonitorLocationResponseFromVTPool() *UpsertRelayMonitorLocationResponse { + return vtprotoPool_UpsertRelayMonitorLocationResponse.Get().(*UpsertRelayMonitorLocationResponse) +} func (m *CreateNodeRecordsRequest) SizeVT() (n int) { if m == nil { return 0 @@ -4868,7 +6193,14 @@ func (m *ExecutionNodeStatus) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Capabilities = append(m.Capabilities, &ExecutionNodeStatus_Capability{}) + if len(m.Capabilities) == cap(m.Capabilities) { + m.Capabilities = append(m.Capabilities, &ExecutionNodeStatus_Capability{}) + } else { + m.Capabilities = m.Capabilities[:len(m.Capabilities)+1] + if m.Capabilities[len(m.Capabilities)-1] == nil { + m.Capabilities[len(m.Capabilities)-1] = &ExecutionNodeStatus_Capability{} + } + } if err := m.Capabilities[len(m.Capabilities)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -5041,7 +6373,7 @@ func (m *ExecutionNodeStatus) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ForkId == nil { - m.ForkId = &ExecutionNodeStatus_ForkID{} + m.ForkId = ExecutionNodeStatus_ForkIDFromVTPool() } if err := m.ForkId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5128,7 +6460,7 @@ func (m *CreateExecutionNodeRecordStatusRequest) UnmarshalVT(dAtA []byte) error return io.ErrUnexpectedEOF } if m.Status == nil { - m.Status = &ExecutionNodeStatus{} + m.Status = ExecutionNodeStatusFromVTPool() } if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -5387,7 +6719,14 @@ func (m *CoordinateExecutionNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.NodeRecords = append(m.NodeRecords, &CoordinatedNodeRecord{}) + if len(m.NodeRecords) == cap(m.NodeRecords) { + m.NodeRecords = append(m.NodeRecords, &CoordinatedNodeRecord{}) + } else { + m.NodeRecords = m.NodeRecords[:len(m.NodeRecords)+1] + if m.NodeRecords[len(m.NodeRecords)-1] == nil { + m.NodeRecords[len(m.NodeRecords)-1] = &CoordinatedNodeRecord{} + } + } if err := m.NodeRecords[len(m.NodeRecords)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -5463,7 +6802,7 @@ func (m *CoordinateExecutionNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.NetworkIds) == 0 { + if elementCount != 0 && len(m.NetworkIds) == 0 && cap(m.NetworkIds) < elementCount { m.NetworkIds = make([]uint64, 0, elementCount) } for iNdEx < postIndex { @@ -6427,7 +7766,7 @@ func (m *CreateConsensusNodeRecordStatusRequest) UnmarshalVT(dAtA []byte) error return io.ErrUnexpectedEOF } if m.Status == nil { - m.Status = &ConsensusNodeStatus{} + m.Status = ConsensusNodeStatusFromVTPool() } if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6564,7 +7903,14 @@ func (m *CreateConsensusNodeRecordStatusesRequest) UnmarshalVT(dAtA []byte) erro if postIndex > l { return io.ErrUnexpectedEOF } - m.Statuses = append(m.Statuses, &ConsensusNodeStatus{}) + if len(m.Statuses) == cap(m.Statuses) { + m.Statuses = append(m.Statuses, &ConsensusNodeStatus{}) + } else { + m.Statuses = m.Statuses[:len(m.Statuses)+1] + if m.Statuses[len(m.Statuses)-1] == nil { + m.Statuses[len(m.Statuses)-1] = &ConsensusNodeStatus{} + } + } if err := m.Statuses[len(m.Statuses)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -6700,7 +8046,14 @@ func (m *CoordinateConsensusNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.NodeRecords = append(m.NodeRecords, &CoordinatedNodeRecord{}) + if len(m.NodeRecords) == cap(m.NodeRecords) { + m.NodeRecords = append(m.NodeRecords, &CoordinatedNodeRecord{}) + } else { + m.NodeRecords = m.NodeRecords[:len(m.NodeRecords)+1] + if m.NodeRecords[len(m.NodeRecords)-1] == nil { + m.NodeRecords[len(m.NodeRecords)-1] = &CoordinatedNodeRecord{} + } + } if err := m.NodeRecords[len(m.NodeRecords)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -6776,7 +8129,7 @@ func (m *CoordinateConsensusNodeRecordsRequest) UnmarshalVT(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.NetworkIds) == 0 { + if elementCount != 0 && len(m.NetworkIds) == 0 && cap(m.NetworkIds) < elementCount { m.NetworkIds = make([]uint64, 0, elementCount) } for iNdEx < postIndex { @@ -7101,7 +8454,7 @@ func (m *GetDiscoveryNodeRecordRequest) UnmarshalVT(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.NetworkIds) == 0 { + if elementCount != 0 && len(m.NetworkIds) == 0 && cap(m.NetworkIds) < elementCount { m.NetworkIds = make([]uint64, 0, elementCount) } for iNdEx < postIndex { @@ -7343,7 +8696,7 @@ func (m *GetDiscoveryExecutionNodeRecordRequest) UnmarshalVT(dAtA []byte) error } } elementCount = count - if elementCount != 0 && len(m.NetworkIds) == 0 { + if elementCount != 0 && len(m.NetworkIds) == 0 && cap(m.NetworkIds) < elementCount { m.NetworkIds = make([]uint64, 0, elementCount) } for iNdEx < postIndex { @@ -7585,7 +8938,7 @@ func (m *GetDiscoveryConsensusNodeRecordRequest) UnmarshalVT(dAtA []byte) error } } elementCount = count - if elementCount != 0 && len(m.NetworkIds) == 0 { + if elementCount != 0 && len(m.NetworkIds) == 0 && cap(m.NetworkIds) < elementCount { m.NetworkIds = make([]uint64, 0, elementCount) } for iNdEx < postIndex { @@ -7894,7 +9247,7 @@ func (m *CannonLocationEthV2BeaconBlockVoluntaryExit) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -7981,7 +9334,7 @@ func (m *CannonLocationEthV2BeaconBlockProposerSlashing) UnmarshalVT(dAtA []byte return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8068,7 +9421,7 @@ func (m *CannonLocationEthV2BeaconBlockDeposit) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8155,7 +9508,7 @@ func (m *CannonLocationEthV2BeaconBlockAttesterSlashing) UnmarshalVT(dAtA []byte return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8242,7 +9595,7 @@ func (m *CannonLocationEthV2BeaconBlockBlsToExecutionChange) UnmarshalVT(dAtA [] return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8329,7 +9682,7 @@ func (m *CannonLocationEthV2BeaconBlockExecutionTransaction) UnmarshalVT(dAtA [] return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8416,7 +9769,7 @@ func (m *CannonLocationEthV2BeaconBlockWithdrawal) UnmarshalVT(dAtA []byte) erro return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8503,7 +9856,7 @@ func (m *CannonLocationEthV2BeaconBlock) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8590,7 +9943,7 @@ func (m *CannonLocationEthV1BeaconBlobSidecar) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8677,7 +10030,7 @@ func (m *CannonLocationEthV1BeaconProposerDuty) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8764,7 +10117,7 @@ func (m *CannonLocationEthV2BeaconBlockElaboratedAttestation) UnmarshalVT(dAtA [ return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8851,7 +10204,7 @@ func (m *CannonLocationEthV1BeaconValidators) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8938,7 +10291,7 @@ func (m *CannonLocationEthV1BeaconCommittee) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9025,7 +10378,7 @@ func (m *CannonLocationEthV1BeaconSyncCommittee) UnmarshalVT(dAtA []byte) error return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9112,7 +10465,7 @@ func (m *CannonLocationEthV2BeaconBlockSyncAggregate) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.BackfillingCheckpointMarker == nil { - m.BackfillingCheckpointMarker = &BackfillingCheckpointMarker{} + m.BackfillingCheckpointMarker = BackfillingCheckpointMarkerFromVTPool() } if err := m.BackfillingCheckpointMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9254,7 +10607,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockVoluntaryExit{} + v := CannonLocationEthV2BeaconBlockVoluntaryExitFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9295,7 +10648,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockProposerSlashing{} + v := CannonLocationEthV2BeaconBlockProposerSlashingFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9336,7 +10689,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockDeposit{} + v := CannonLocationEthV2BeaconBlockDepositFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9377,7 +10730,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockAttesterSlashing{} + v := CannonLocationEthV2BeaconBlockAttesterSlashingFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9418,7 +10771,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockBlsToExecutionChange{} + v := CannonLocationEthV2BeaconBlockBlsToExecutionChangeFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9459,7 +10812,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockExecutionTransaction{} + v := CannonLocationEthV2BeaconBlockExecutionTransactionFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9500,7 +10853,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockWithdrawal{} + v := CannonLocationEthV2BeaconBlockWithdrawalFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9541,7 +10894,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlock{} + v := CannonLocationEthV2BeaconBlockFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9582,7 +10935,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV1BeaconBlobSidecar{} + v := CannonLocationEthV1BeaconBlobSidecarFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9623,7 +10976,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV1BeaconProposerDuty{} + v := CannonLocationEthV1BeaconProposerDutyFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9664,7 +11017,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockElaboratedAttestation{} + v := CannonLocationEthV2BeaconBlockElaboratedAttestationFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9705,7 +11058,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV1BeaconValidators{} + v := CannonLocationEthV1BeaconValidatorsFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9746,7 +11099,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV1BeaconCommittee{} + v := CannonLocationEthV1BeaconCommitteeFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9787,7 +11140,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV1BeaconSyncCommittee{} + v := CannonLocationEthV1BeaconSyncCommitteeFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9828,7 +11181,7 @@ func (m *CannonLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &CannonLocationEthV2BeaconBlockSyncAggregate{} + v := CannonLocationEthV2BeaconBlockSyncAggregateFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10018,7 +11371,7 @@ func (m *GetCannonLocationResponse) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Location == nil { - m.Location = &CannonLocation{} + m.Location = CannonLocationFromVTPool() } if err := m.Location.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10105,7 +11458,7 @@ func (m *UpsertCannonLocationRequest) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Location == nil { - m.Location = &CannonLocation{} + m.Location = CannonLocationFromVTPool() } if err := m.Location.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10313,7 +11666,7 @@ func (m *RelayMonitorLocationBidTrace) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SlotMarker == nil { - m.SlotMarker = &RelayMonitorSlotMarker{} + m.SlotMarker = RelayMonitorSlotMarkerFromVTPool() } if err := m.SlotMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10400,7 +11753,7 @@ func (m *RelayMonitorLocationPayloadDelivered) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SlotMarker == nil { - m.SlotMarker = &RelayMonitorSlotMarker{} + m.SlotMarker = RelayMonitorSlotMarkerFromVTPool() } if err := m.SlotMarker.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10606,7 +11959,7 @@ func (m *RelayMonitorLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &RelayMonitorLocationBidTrace{} + v := RelayMonitorLocationBidTraceFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10647,7 +12000,7 @@ func (m *RelayMonitorLocation) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &RelayMonitorLocationPayloadDelivered{} + v := RelayMonitorLocationPayloadDeliveredFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10901,7 +12254,7 @@ func (m *GetRelayMonitorLocationResponse) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Location == nil { - m.Location = &RelayMonitorLocation{} + m.Location = RelayMonitorLocationFromVTPool() } if err := m.Location.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10988,7 +12341,7 @@ func (m *UpsertRelayMonitorLocationRequest) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Location == nil { - m.Location = &RelayMonitorLocation{} + m.Location = RelayMonitorLocationFromVTPool() } if err := m.Location.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err diff --git a/pkg/proto/xatu/event_ingester_vtproto.pb.go b/pkg/proto/xatu/event_ingester_vtproto.pb.go index 189cf394..8b75f726 100644 --- a/pkg/proto/xatu/event_ingester_vtproto.pb.go +++ b/pkg/proto/xatu/event_ingester_vtproto.pb.go @@ -21,6 +21,7 @@ import ( wrapperspb1 "google.golang.org/protobuf/types/known/wrapperspb" io "io" math "math" + sync "sync" ) const ( @@ -12628,6 +12629,3702 @@ func (m *DecoratedEvent_ExecutionBlockMetrics) MarshalToSizedBufferVT(dAtA []byt } return len(dAtA) - i, nil } + +var vtprotoPool_CreateEventsRequest = sync.Pool{ + New: func() interface{} { + return &CreateEventsRequest{} + }, +} + +func (m *CreateEventsRequest) ResetVT() { + if m != nil { + for _, mm := range m.Events { + mm.ResetVT() + } + f0 := m.Events[:0] + m.Reset() + m.Events = f0 + } +} +func (m *CreateEventsRequest) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateEventsRequest.Put(m) + } +} +func CreateEventsRequestFromVTPool() *CreateEventsRequest { + return vtprotoPool_CreateEventsRequest.Get().(*CreateEventsRequest) +} + +var vtprotoPool_CreateEventsResponse = sync.Pool{ + New: func() interface{} { + return &CreateEventsResponse{} + }, +} + +func (m *CreateEventsResponse) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *CreateEventsResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_CreateEventsResponse.Put(m) + } +} +func CreateEventsResponseFromVTPool() *CreateEventsResponse { + return vtprotoPool_CreateEventsResponse.Get().(*CreateEventsResponse) +} + +var vtprotoPool_Epoch = sync.Pool{ + New: func() interface{} { + return &Epoch{} + }, +} + +func (m *Epoch) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Epoch) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Epoch.Put(m) + } +} +func EpochFromVTPool() *Epoch { + return vtprotoPool_Epoch.Get().(*Epoch) +} + +var vtprotoPool_EpochV2 = sync.Pool{ + New: func() interface{} { + return &EpochV2{} + }, +} + +func (m *EpochV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *EpochV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_EpochV2.Put(m) + } +} +func EpochV2FromVTPool() *EpochV2 { + return vtprotoPool_EpochV2.Get().(*EpochV2) +} + +var vtprotoPool_Slot = sync.Pool{ + New: func() interface{} { + return &Slot{} + }, +} + +func (m *Slot) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Slot) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Slot.Put(m) + } +} +func SlotFromVTPool() *Slot { + return vtprotoPool_Slot.Get().(*Slot) +} + +var vtprotoPool_SlotV2 = sync.Pool{ + New: func() interface{} { + return &SlotV2{} + }, +} + +func (m *SlotV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *SlotV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SlotV2.Put(m) + } +} +func SlotV2FromVTPool() *SlotV2 { + return vtprotoPool_SlotV2.Get().(*SlotV2) +} + +var vtprotoPool_ForkID = sync.Pool{ + New: func() interface{} { + return &ForkID{} + }, +} + +func (m *ForkID) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ForkID) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ForkID.Put(m) + } +} +func ForkIDFromVTPool() *ForkID { + return vtprotoPool_ForkID.Get().(*ForkID) +} + +var vtprotoPool_Propagation = sync.Pool{ + New: func() interface{} { + return &Propagation{} + }, +} + +func (m *Propagation) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Propagation) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Propagation.Put(m) + } +} +func PropagationFromVTPool() *Propagation { + return vtprotoPool_Propagation.Get().(*Propagation) +} + +var vtprotoPool_PropagationV2 = sync.Pool{ + New: func() interface{} { + return &PropagationV2{} + }, +} + +func (m *PropagationV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *PropagationV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_PropagationV2.Put(m) + } +} +func PropagationV2FromVTPool() *PropagationV2 { + return vtprotoPool_PropagationV2.Get().(*PropagationV2) +} + +var vtprotoPool_AttestingValidator = sync.Pool{ + New: func() interface{} { + return &AttestingValidator{} + }, +} + +func (m *AttestingValidator) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *AttestingValidator) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AttestingValidator.Put(m) + } +} +func AttestingValidatorFromVTPool() *AttestingValidator { + return vtprotoPool_AttestingValidator.Get().(*AttestingValidator) +} + +var vtprotoPool_AttestingValidatorV2 = sync.Pool{ + New: func() interface{} { + return &AttestingValidatorV2{} + }, +} + +func (m *AttestingValidatorV2) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *AttestingValidatorV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_AttestingValidatorV2.Put(m) + } +} +func AttestingValidatorV2FromVTPool() *AttestingValidatorV2 { + return vtprotoPool_AttestingValidatorV2.Get().(*AttestingValidatorV2) +} + +var vtprotoPool_DebugForkChoiceReorg = sync.Pool{ + New: func() interface{} { + return &DebugForkChoiceReorg{} + }, +} + +func (m *DebugForkChoiceReorg) ResetVT() { + if m != nil { + m.Before.ReturnToVTPool() + m.After.ReturnToVTPool() + m.Event.ReturnToVTPool() + m.Reset() + } +} +func (m *DebugForkChoiceReorg) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DebugForkChoiceReorg.Put(m) + } +} +func DebugForkChoiceReorgFromVTPool() *DebugForkChoiceReorg { + return vtprotoPool_DebugForkChoiceReorg.Get().(*DebugForkChoiceReorg) +} + +var vtprotoPool_DebugForkChoiceReorgV2 = sync.Pool{ + New: func() interface{} { + return &DebugForkChoiceReorgV2{} + }, +} + +func (m *DebugForkChoiceReorgV2) ResetVT() { + if m != nil { + m.Before.ReturnToVTPool() + m.After.ReturnToVTPool() + m.Event.ReturnToVTPool() + m.Reset() + } +} +func (m *DebugForkChoiceReorgV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DebugForkChoiceReorgV2.Put(m) + } +} +func DebugForkChoiceReorgV2FromVTPool() *DebugForkChoiceReorgV2 { + return vtprotoPool_DebugForkChoiceReorgV2.Get().(*DebugForkChoiceReorgV2) +} + +var vtprotoPool_Validators = sync.Pool{ + New: func() interface{} { + return &Validators{} + }, +} + +func (m *Validators) ResetVT() { + if m != nil { + for _, mm := range m.Validators { + mm.ResetVT() + } + f0 := m.Validators[:0] + m.Reset() + m.Validators = f0 + } +} +func (m *Validators) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Validators.Put(m) + } +} +func ValidatorsFromVTPool() *Validators { + return vtprotoPool_Validators.Get().(*Validators) +} + +var vtprotoPool_SyncCommitteeData = sync.Pool{ + New: func() interface{} { + return &SyncCommitteeData{} + }, +} + +func (m *SyncCommitteeData) ResetVT() { + if m != nil { + m.SyncCommittee.ReturnToVTPool() + m.Reset() + } +} +func (m *SyncCommitteeData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SyncCommitteeData.Put(m) + } +} +func SyncCommitteeDataFromVTPool() *SyncCommitteeData { + return vtprotoPool_SyncCommitteeData.Get().(*SyncCommitteeData) +} + +var vtprotoPool_SyncAggregateData = sync.Pool{ + New: func() interface{} { + return &SyncAggregateData{} + }, +} + +func (m *SyncAggregateData) ResetVT() { + if m != nil { + for _, mm := range m.ValidatorsParticipated { + mm.Reset() + } + f0 := m.ValidatorsParticipated[:0] + for _, mm := range m.ValidatorsMissed { + mm.Reset() + } + f1 := m.ValidatorsMissed[:0] + m.Reset() + m.ValidatorsParticipated = f0 + m.ValidatorsMissed = f1 + } +} +func (m *SyncAggregateData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_SyncAggregateData.Put(m) + } +} +func SyncAggregateDataFromVTPool() *SyncAggregateData { + return vtprotoPool_SyncAggregateData.Get().(*SyncAggregateData) +} + +var vtprotoPool_BlockIdentifier = sync.Pool{ + New: func() interface{} { + return &BlockIdentifier{} + }, +} + +func (m *BlockIdentifier) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *BlockIdentifier) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_BlockIdentifier.Put(m) + } +} +func BlockIdentifierFromVTPool() *BlockIdentifier { + return vtprotoPool_BlockIdentifier.Get().(*BlockIdentifier) +} + +var vtprotoPool_ExecutionStateSize = sync.Pool{ + New: func() interface{} { + return &ExecutionStateSize{} + }, +} + +func (m *ExecutionStateSize) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionStateSize) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionStateSize.Put(m) + } +} +func ExecutionStateSizeFromVTPool() *ExecutionStateSize { + return vtprotoPool_ExecutionStateSize.Get().(*ExecutionStateSize) +} + +var vtprotoPool_ConsensusEngineAPINewPayload = sync.Pool{ + New: func() interface{} { + return &ConsensusEngineAPINewPayload{} + }, +} + +func (m *ConsensusEngineAPINewPayload) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ConsensusEngineAPINewPayload) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ConsensusEngineAPINewPayload.Put(m) + } +} +func ConsensusEngineAPINewPayloadFromVTPool() *ConsensusEngineAPINewPayload { + return vtprotoPool_ConsensusEngineAPINewPayload.Get().(*ConsensusEngineAPINewPayload) +} + +var vtprotoPool_ConsensusEngineAPIGetBlobs = sync.Pool{ + New: func() interface{} { + return &ConsensusEngineAPIGetBlobs{} + }, +} + +func (m *ConsensusEngineAPIGetBlobs) ResetVT() { + if m != nil { + f0 := m.VersionedHashes[:0] + m.Reset() + m.VersionedHashes = f0 + } +} +func (m *ConsensusEngineAPIGetBlobs) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ConsensusEngineAPIGetBlobs.Put(m) + } +} +func ConsensusEngineAPIGetBlobsFromVTPool() *ConsensusEngineAPIGetBlobs { + return vtprotoPool_ConsensusEngineAPIGetBlobs.Get().(*ConsensusEngineAPIGetBlobs) +} + +var vtprotoPool_ExecutionEngineNewPayload = sync.Pool{ + New: func() interface{} { + return &ExecutionEngineNewPayload{} + }, +} + +func (m *ExecutionEngineNewPayload) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionEngineNewPayload) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionEngineNewPayload.Put(m) + } +} +func ExecutionEngineNewPayloadFromVTPool() *ExecutionEngineNewPayload { + return vtprotoPool_ExecutionEngineNewPayload.Get().(*ExecutionEngineNewPayload) +} + +var vtprotoPool_ExecutionEngineGetBlobs = sync.Pool{ + New: func() interface{} { + return &ExecutionEngineGetBlobs{} + }, +} + +func (m *ExecutionEngineGetBlobs) ResetVT() { + if m != nil { + f0 := m.VersionedHashes[:0] + for _, mm := range m.ReturnedBlobIndexes { + mm.Reset() + } + f1 := m.ReturnedBlobIndexes[:0] + m.Reset() + m.VersionedHashes = f0 + m.ReturnedBlobIndexes = f1 + } +} +func (m *ExecutionEngineGetBlobs) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionEngineGetBlobs.Put(m) + } +} +func ExecutionEngineGetBlobsFromVTPool() *ExecutionEngineGetBlobs { + return vtprotoPool_ExecutionEngineGetBlobs.Get().(*ExecutionEngineGetBlobs) +} + +var vtprotoPool_ClientMeta_Ethereum_Network = sync.Pool{ + New: func() interface{} { + return &ClientMeta_Ethereum_Network{} + }, +} + +func (m *ClientMeta_Ethereum_Network) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ClientMeta_Ethereum_Network) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_Ethereum_Network.Put(m) + } +} +func ClientMeta_Ethereum_NetworkFromVTPool() *ClientMeta_Ethereum_Network { + return vtprotoPool_ClientMeta_Ethereum_Network.Get().(*ClientMeta_Ethereum_Network) +} + +var vtprotoPool_ClientMeta_Ethereum_Execution = sync.Pool{ + New: func() interface{} { + return &ClientMeta_Ethereum_Execution{} + }, +} + +func (m *ClientMeta_Ethereum_Execution) ResetVT() { + if m != nil { + m.ForkId.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_Ethereum_Execution) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_Ethereum_Execution.Put(m) + } +} +func ClientMeta_Ethereum_ExecutionFromVTPool() *ClientMeta_Ethereum_Execution { + return vtprotoPool_ClientMeta_Ethereum_Execution.Get().(*ClientMeta_Ethereum_Execution) +} + +var vtprotoPool_ClientMeta_Ethereum_Consensus = sync.Pool{ + New: func() interface{} { + return &ClientMeta_Ethereum_Consensus{} + }, +} + +func (m *ClientMeta_Ethereum_Consensus) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ClientMeta_Ethereum_Consensus) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_Ethereum_Consensus.Put(m) + } +} +func ClientMeta_Ethereum_ConsensusFromVTPool() *ClientMeta_Ethereum_Consensus { + return vtprotoPool_ClientMeta_Ethereum_Consensus.Get().(*ClientMeta_Ethereum_Consensus) +} + +var vtprotoPool_ClientMeta_Ethereum = sync.Pool{ + New: func() interface{} { + return &ClientMeta_Ethereum{} + }, +} + +func (m *ClientMeta_Ethereum) ResetVT() { + if m != nil { + m.Network.ReturnToVTPool() + m.Execution.ReturnToVTPool() + m.Consensus.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_Ethereum) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_Ethereum.Put(m) + } +} +func ClientMeta_EthereumFromVTPool() *ClientMeta_Ethereum { + return vtprotoPool_ClientMeta_Ethereum.Get().(*ClientMeta_Ethereum) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1AttestationSourceData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1AttestationSourceData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1AttestationSourceData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1AttestationSourceData.Put(m) + } +} +func ClientMeta_AdditionalEthV1AttestationSourceDataFromVTPool() *ClientMeta_AdditionalEthV1AttestationSourceData { + return vtprotoPool_ClientMeta_AdditionalEthV1AttestationSourceData.Get().(*ClientMeta_AdditionalEthV1AttestationSourceData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1AttestationSourceV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1AttestationSourceV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1AttestationSourceV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1AttestationSourceV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1AttestationSourceV2DataFromVTPool() *ClientMeta_AdditionalEthV1AttestationSourceV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1AttestationSourceV2Data.Get().(*ClientMeta_AdditionalEthV1AttestationSourceV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1AttestationTargetData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1AttestationTargetData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1AttestationTargetData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1AttestationTargetData.Put(m) + } +} +func ClientMeta_AdditionalEthV1AttestationTargetDataFromVTPool() *ClientMeta_AdditionalEthV1AttestationTargetData { + return vtprotoPool_ClientMeta_AdditionalEthV1AttestationTargetData.Get().(*ClientMeta_AdditionalEthV1AttestationTargetData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1AttestationTargetV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1AttestationTargetV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1AttestationTargetV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1AttestationTargetV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1AttestationTargetV2DataFromVTPool() *ClientMeta_AdditionalEthV1AttestationTargetV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1AttestationTargetV2Data.Get().(*ClientMeta_AdditionalEthV1AttestationTargetV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsAttestationData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsAttestationData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationData) ResetVT() { + if m != nil { + m.Source.ReturnToVTPool() + m.Target.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.AttestingValidator.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsAttestationData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsAttestationData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsAttestationDataFromVTPool() *ClientMeta_AdditionalEthV1EventsAttestationData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsAttestationData.Get().(*ClientMeta_AdditionalEthV1EventsAttestationData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsAttestationV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsAttestationV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) ResetVT() { + if m != nil { + m.Source.ReturnToVTPool() + m.Target.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.AttestingValidator.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsAttestationV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsAttestationV2DataFromVTPool() *ClientMeta_AdditionalEthV1EventsAttestationV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsAttestationV2Data.Get().(*ClientMeta_AdditionalEthV1EventsAttestationV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsHeadData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsHeadData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsHeadData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsHeadData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsHeadDataFromVTPool() *ClientMeta_AdditionalEthV1EventsHeadData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsHeadData.Get().(*ClientMeta_AdditionalEthV1EventsHeadData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsHeadV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsHeadV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsHeadV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsHeadV2DataFromVTPool() *ClientMeta_AdditionalEthV1EventsHeadV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsHeadV2Data.Get().(*ClientMeta_AdditionalEthV1EventsHeadV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsBlockData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsBlockData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsBlockDataFromVTPool() *ClientMeta_AdditionalEthV1EventsBlockData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockData.Get().(*ClientMeta_AdditionalEthV1EventsBlockData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsBlockV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsBlockV2DataFromVTPool() *ClientMeta_AdditionalEthV1EventsBlockV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockV2Data.Get().(*ClientMeta_AdditionalEthV1EventsBlockV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockGossipData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsBlockGossipData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockGossipData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsBlockGossipDataFromVTPool() *ClientMeta_AdditionalEthV1EventsBlockGossipData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsBlockGossipData.Get().(*ClientMeta_AdditionalEthV1EventsBlockGossipData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsVoluntaryExitData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsVoluntaryExitData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsVoluntaryExitData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsVoluntaryExitDataFromVTPool() *ClientMeta_AdditionalEthV1EventsVoluntaryExitData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsVoluntaryExitData.Get().(*ClientMeta_AdditionalEthV1EventsVoluntaryExitData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsVoluntaryExitV2DataFromVTPool() *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data.Get().(*ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsFinalizedCheckpointDataFromVTPool() *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData.Get().(*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2DataFromVTPool() *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data.Get().(*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsChainReorgData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsChainReorgData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsChainReorgData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsChainReorgDataFromVTPool() *ClientMeta_AdditionalEthV1EventsChainReorgData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsChainReorgData.Get().(*ClientMeta_AdditionalEthV1EventsChainReorgData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsChainReorgV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsChainReorgV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsChainReorgV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsChainReorgV2DataFromVTPool() *ClientMeta_AdditionalEthV1EventsChainReorgV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsChainReorgV2Data.Get().(*ClientMeta_AdditionalEthV1EventsChainReorgV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsContributionAndProofContributionDataFromVTPool() *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData.Get().(*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2DataFromVTPool() *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data.Get().(*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsContributionAndProofData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofData) ResetVT() { + if m != nil { + m.Contribution.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsContributionAndProofDataFromVTPool() *ClientMeta_AdditionalEthV1EventsContributionAndProofData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofData.Get().(*ClientMeta_AdditionalEthV1EventsContributionAndProofData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) ResetVT() { + if m != nil { + m.Contribution.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsContributionAndProofV2DataFromVTPool() *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data.Get().(*ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) +} + +var vtprotoPool_ClientMeta_ForkChoiceSnapshot = sync.Pool{ + New: func() interface{} { + return &ClientMeta_ForkChoiceSnapshot{} + }, +} + +func (m *ClientMeta_ForkChoiceSnapshot) ResetVT() { + if m != nil { + m.RequestEpoch.ReturnToVTPool() + m.RequestSlot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_ForkChoiceSnapshot) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_ForkChoiceSnapshot.Put(m) + } +} +func ClientMeta_ForkChoiceSnapshotFromVTPool() *ClientMeta_ForkChoiceSnapshot { + return vtprotoPool_ClientMeta_ForkChoiceSnapshot.Get().(*ClientMeta_ForkChoiceSnapshot) +} + +var vtprotoPool_ClientMeta_ForkChoiceSnapshotV2 = sync.Pool{ + New: func() interface{} { + return &ClientMeta_ForkChoiceSnapshotV2{} + }, +} + +func (m *ClientMeta_ForkChoiceSnapshotV2) ResetVT() { + if m != nil { + m.RequestEpoch.ReturnToVTPool() + m.RequestSlot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_ForkChoiceSnapshotV2) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_ForkChoiceSnapshotV2.Put(m) + } +} +func ClientMeta_ForkChoiceSnapshotV2FromVTPool() *ClientMeta_ForkChoiceSnapshotV2 { + return vtprotoPool_ClientMeta_ForkChoiceSnapshotV2.Get().(*ClientMeta_ForkChoiceSnapshotV2) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1DebugForkChoiceData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceData) ResetVT() { + if m != nil { + m.Snapshot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceData.Put(m) + } +} +func ClientMeta_AdditionalEthV1DebugForkChoiceDataFromVTPool() *ClientMeta_AdditionalEthV1DebugForkChoiceData { + return vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceData.Get().(*ClientMeta_AdditionalEthV1DebugForkChoiceData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1DebugForkChoiceV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) ResetVT() { + if m != nil { + m.Snapshot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1DebugForkChoiceV2DataFromVTPool() *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceV2Data.Get().(*ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) ResetVT() { + if m != nil { + m.Before.ReturnToVTPool() + m.After.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData.Put(m) + } +} +func ClientMeta_AdditionalEthV1DebugForkChoiceReOrgDataFromVTPool() *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData { + return vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData.Get().(*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) ResetVT() { + if m != nil { + m.Before.ReturnToVTPool() + m.After.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2DataFromVTPool() *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data.Get().(*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1BeaconCommitteeData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1BeaconCommitteeData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1BeaconCommitteeData.Put(m) + } +} +func ClientMeta_AdditionalEthV1BeaconCommitteeDataFromVTPool() *ClientMeta_AdditionalEthV1BeaconCommitteeData { + return vtprotoPool_ClientMeta_AdditionalEthV1BeaconCommitteeData.Get().(*ClientMeta_AdditionalEthV1BeaconCommitteeData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1BeaconSyncCommitteeData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1BeaconSyncCommitteeData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1BeaconSyncCommitteeData.Put(m) + } +} +func ClientMeta_AdditionalEthV1BeaconSyncCommitteeDataFromVTPool() *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData { + return vtprotoPool_ClientMeta_AdditionalEthV1BeaconSyncCommitteeData.Get().(*ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) +} + +var vtprotoPool_ClientMeta_AdditionalMempoolTransactionData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalMempoolTransactionData{} + }, +} + +func (m *ClientMeta_AdditionalMempoolTransactionData) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ClientMeta_AdditionalMempoolTransactionData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalMempoolTransactionData.Put(m) + } +} +func ClientMeta_AdditionalMempoolTransactionDataFromVTPool() *ClientMeta_AdditionalMempoolTransactionData { + return vtprotoPool_ClientMeta_AdditionalMempoolTransactionData.Get().(*ClientMeta_AdditionalMempoolTransactionData) +} + +var vtprotoPool_ClientMeta_AdditionalMempoolTransactionV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalMempoolTransactionV2Data{} + }, +} + +func (m *ClientMeta_AdditionalMempoolTransactionV2Data) ResetVT() { + if m != nil { + f0 := m.BlobHashes[:0] + m.Reset() + m.BlobHashes = f0 + } +} +func (m *ClientMeta_AdditionalMempoolTransactionV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalMempoolTransactionV2Data.Put(m) + } +} +func ClientMeta_AdditionalMempoolTransactionV2DataFromVTPool() *ClientMeta_AdditionalMempoolTransactionV2Data { + return vtprotoPool_ClientMeta_AdditionalMempoolTransactionV2Data.Get().(*ClientMeta_AdditionalMempoolTransactionV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockV2Data = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockV2Data{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockV2Data.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockV2DataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockV2Data { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockV2Data.Get().(*ClientMeta_AdditionalEthV2BeaconBlockV2Data) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockDepositData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockDepositData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockDepositData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockDepositData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockDepositData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockDepositDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockDepositData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockDepositData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockDepositData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockWithdrawalDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) +} + +var vtprotoPool_ClientMeta_AttestationDataSnapshot = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AttestationDataSnapshot{} + }, +} + +func (m *ClientMeta_AttestationDataSnapshot) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ClientMeta_AttestationDataSnapshot) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AttestationDataSnapshot.Put(m) + } +} +func ClientMeta_AttestationDataSnapshotFromVTPool() *ClientMeta_AttestationDataSnapshot { + return vtprotoPool_ClientMeta_AttestationDataSnapshot.Get().(*ClientMeta_AttestationDataSnapshot) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1ValidatorAttestationDataData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1ValidatorAttestationDataData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) ResetVT() { + if m != nil { + m.Source.ReturnToVTPool() + m.Target.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Snapshot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1ValidatorAttestationDataData.Put(m) + } +} +func ClientMeta_AdditionalEthV1ValidatorAttestationDataDataFromVTPool() *ClientMeta_AdditionalEthV1ValidatorAttestationDataData { + return vtprotoPool_ClientMeta_AdditionalEthV1ValidatorAttestationDataData.Get().(*ClientMeta_AdditionalEthV1ValidatorAttestationDataData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsBlobSidecarData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsBlobSidecarData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsBlobSidecarData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsBlobSidecarDataFromVTPool() *ClientMeta_AdditionalEthV1EventsBlobSidecarData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsBlobSidecarData.Get().(*ClientMeta_AdditionalEthV1EventsBlobSidecarData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1EventsDataColumnSidecarData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1EventsDataColumnSidecarData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1EventsDataColumnSidecarData.Put(m) + } +} +func ClientMeta_AdditionalEthV1EventsDataColumnSidecarDataFromVTPool() *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData { + return vtprotoPool_ClientMeta_AdditionalEthV1EventsDataColumnSidecarData.Get().(*ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1BeaconBlobSidecarData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1BeaconBlobSidecarData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1BeaconBlobSidecarData.Put(m) + } +} +func ClientMeta_AdditionalEthV1BeaconBlobSidecarDataFromVTPool() *ClientMeta_AdditionalEthV1BeaconBlobSidecarData { + return vtprotoPool_ClientMeta_AdditionalEthV1BeaconBlobSidecarData.Get().(*ClientMeta_AdditionalEthV1BeaconBlobSidecarData) +} + +var vtprotoPool_ClientMeta_AdditionalBeaconP2PAttestationData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalBeaconP2PAttestationData{} + }, +} + +func (m *ClientMeta_AdditionalBeaconP2PAttestationData) ResetVT() { + if m != nil { + m.Source.ReturnToVTPool() + m.Target.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.AttestingValidator.ReturnToVTPool() + m.Peer.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalBeaconP2PAttestationData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalBeaconP2PAttestationData.Put(m) + } +} +func ClientMeta_AdditionalBeaconP2PAttestationDataFromVTPool() *ClientMeta_AdditionalBeaconP2PAttestationData { + return vtprotoPool_ClientMeta_AdditionalBeaconP2PAttestationData.Get().(*ClientMeta_AdditionalBeaconP2PAttestationData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1ProposerDutyData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1ProposerDutyData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1ProposerDutyData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1ProposerDutyData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1ProposerDutyData.Put(m) + } +} +func ClientMeta_AdditionalEthV1ProposerDutyDataFromVTPool() *ClientMeta_AdditionalEthV1ProposerDutyData { + return vtprotoPool_ClientMeta_AdditionalEthV1ProposerDutyData.Get().(*ClientMeta_AdditionalEthV1ProposerDutyData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData{} + }, +} + +func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) ResetVT() { + if m != nil { + m.Block.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Source.ReturnToVTPool() + m.Target.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData.Put(m) + } +} +func ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationDataFromVTPool() *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData { + return vtprotoPool_ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData.Get().(*ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceAddPeerData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceAddPeerData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceAddPeerData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceAddPeerData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceAddPeerData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceAddPeerDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceAddPeerData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceAddPeerData.Get().(*ClientMeta_AdditionalLibP2PTraceAddPeerData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRemovePeerData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRemovePeerData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRemovePeerData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRemovePeerData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRemovePeerData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRemovePeerDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRemovePeerData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRemovePeerData.Get().(*ClientMeta_AdditionalLibP2PTraceRemovePeerData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRecvRPCData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRecvRPCData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRecvRPCData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRecvRPCData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRecvRPCData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRecvRPCDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRecvRPCData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRecvRPCData.Get().(*ClientMeta_AdditionalLibP2PTraceRecvRPCData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceSendRPCData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceSendRPCData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceSendRPCData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceSendRPCData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceSendRPCData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceSendRPCDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceSendRPCData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceSendRPCData.Get().(*ClientMeta_AdditionalLibP2PTraceSendRPCData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceDropRPCData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceDropRPCData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceDropRPCData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceDropRPCData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceDropRPCData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceDropRPCDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceDropRPCData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceDropRPCData.Get().(*ClientMeta_AdditionalLibP2PTraceDropRPCData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData.Get().(*ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData.Get().(*ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData.Get().(*ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData.Get().(*ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData.Get().(*ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceJoinData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceJoinData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceJoinData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceJoinData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceJoinData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceJoinDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceJoinData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceJoinData.Get().(*ClientMeta_AdditionalLibP2PTraceJoinData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceLeaveData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceLeaveData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceLeaveData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceLeaveData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceLeaveData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceLeaveDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceLeaveData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceLeaveData.Get().(*ClientMeta_AdditionalLibP2PTraceLeaveData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceGraftData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceGraftData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceGraftData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceGraftData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceGraftData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceGraftDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceGraftData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceGraftData.Get().(*ClientMeta_AdditionalLibP2PTraceGraftData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTracePruneData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTracePruneData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTracePruneData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTracePruneData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTracePruneData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTracePruneDataFromVTPool() *ClientMeta_AdditionalLibP2PTracePruneData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTracePruneData.Get().(*ClientMeta_AdditionalLibP2PTracePruneData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceDuplicateMessageData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceDuplicateMessageData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceDuplicateMessageData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceDuplicateMessageDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceDuplicateMessageData.Get().(*ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceDeliverMessageData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceDeliverMessageData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceDeliverMessageData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceDeliverMessageData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceDeliverMessageData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceDeliverMessageDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceDeliverMessageData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceDeliverMessageData.Get().(*ClientMeta_AdditionalLibP2PTraceDeliverMessageData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTracePublishMessageData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTracePublishMessageData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTracePublishMessageData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTracePublishMessageData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTracePublishMessageData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTracePublishMessageDataFromVTPool() *ClientMeta_AdditionalLibP2PTracePublishMessageData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTracePublishMessageData.Get().(*ClientMeta_AdditionalLibP2PTracePublishMessageData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRejectMessageData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRejectMessageData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRejectMessageData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRejectMessageData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRejectMessageData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRejectMessageDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRejectMessageData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRejectMessageData.Get().(*ClientMeta_AdditionalLibP2PTraceRejectMessageData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceConnectedData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceConnectedData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceConnectedData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceConnectedData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceConnectedData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceConnectedDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceConnectedData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceConnectedData.Get().(*ClientMeta_AdditionalLibP2PTraceConnectedData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceDisconnectedData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceDisconnectedData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceDisconnectedData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceDisconnectedData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceDisconnectedData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceDisconnectedDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceDisconnectedData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceDisconnectedData.Get().(*ClientMeta_AdditionalLibP2PTraceDisconnectedData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData.Get().(*ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceHandleMetadataData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceHandleMetadataData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceHandleMetadataData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceHandleMetadataDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceHandleMetadataData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceHandleMetadataData.Get().(*ClientMeta_AdditionalLibP2PTraceHandleMetadataData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceHandleStatusData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceHandleStatusData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceHandleStatusData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceHandleStatusData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceHandleStatusData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceHandleStatusDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceHandleStatusData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceHandleStatusData.Get().(*ClientMeta_AdditionalLibP2PTraceHandleStatusData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceIdentifyData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceIdentifyData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceIdentifyData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceIdentifyData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceIdentifyData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceIdentifyDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceIdentifyData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceIdentifyData.Get().(*ClientMeta_AdditionalLibP2PTraceIdentifyData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData.Get().(*ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData.Get().(*ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) ResetVT() { + if m != nil { + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceRPCMetaMessageDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData.Get().(*ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData.Get().(*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData.Get().(*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData.Get().(*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) ResetVT() { + if m != nil { + m.Source.ReturnToVTPool() + m.Target.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.AttestingValidator.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData.Get().(*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData.Get().(*ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData.Get().(*ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) +} + +var vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData{} + }, +} + +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Propagation.ReturnToVTPool() + m.Metadata.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData.Put(m) + } +} +func ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarDataFromVTPool() *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData { + return vtprotoPool_ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData.Get().(*ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1ValidatorsData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1ValidatorsData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1ValidatorsData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1ValidatorsData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1ValidatorsData.Put(m) + } +} +func ClientMeta_AdditionalEthV1ValidatorsDataFromVTPool() *ClientMeta_AdditionalEthV1ValidatorsData { + return vtprotoPool_ClientMeta_AdditionalEthV1ValidatorsData.Get().(*ClientMeta_AdditionalEthV1ValidatorsData) +} + +var vtprotoPool_ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData{} + }, +} + +func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) ResetVT() { + if m != nil { + m.Relay.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData.Put(m) + } +} +func ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionDataFromVTPool() *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData { + return vtprotoPool_ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData.Get().(*ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) +} + +var vtprotoPool_ClientMeta_AdditionalMevRelayPayloadDeliveredData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalMevRelayPayloadDeliveredData{} + }, +} + +func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) ResetVT() { + if m != nil { + m.Relay.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalMevRelayPayloadDeliveredData.Put(m) + } +} +func ClientMeta_AdditionalMevRelayPayloadDeliveredDataFromVTPool() *ClientMeta_AdditionalMevRelayPayloadDeliveredData { + return vtprotoPool_ClientMeta_AdditionalMevRelayPayloadDeliveredData.Get().(*ClientMeta_AdditionalMevRelayPayloadDeliveredData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV3ValidatorBlockData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV3ValidatorBlockData{} + }, +} + +func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV3ValidatorBlockData.Put(m) + } +} +func ClientMeta_AdditionalEthV3ValidatorBlockDataFromVTPool() *ClientMeta_AdditionalEthV3ValidatorBlockData { + return vtprotoPool_ClientMeta_AdditionalEthV3ValidatorBlockData.Get().(*ClientMeta_AdditionalEthV3ValidatorBlockData) +} + +var vtprotoPool_ClientMeta_AdditionalMevRelayValidatorRegistrationData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalMevRelayValidatorRegistrationData{} + }, +} + +func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) ResetVT() { + if m != nil { + m.Relay.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.WallclockSlot.ReturnToVTPool() + m.Epoch.ReturnToVTPool() + m.WallclockEpoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalMevRelayValidatorRegistrationData.Put(m) + } +} +func ClientMeta_AdditionalMevRelayValidatorRegistrationDataFromVTPool() *ClientMeta_AdditionalMevRelayValidatorRegistrationData { + return vtprotoPool_ClientMeta_AdditionalMevRelayValidatorRegistrationData.Get().(*ClientMeta_AdditionalMevRelayValidatorRegistrationData) +} + +var vtprotoPool_ClientMeta_AdditionalNodeRecordConsensusData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalNodeRecordConsensusData{} + }, +} + +func (m *ClientMeta_AdditionalNodeRecordConsensusData) ResetVT() { + if m != nil { + m.FinalizedEpoch.ReturnToVTPool() + m.HeadSlot.ReturnToVTPool() + m.HeadEpoch.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalNodeRecordConsensusData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalNodeRecordConsensusData.Put(m) + } +} +func ClientMeta_AdditionalNodeRecordConsensusDataFromVTPool() *ClientMeta_AdditionalNodeRecordConsensusData { + return vtprotoPool_ClientMeta_AdditionalNodeRecordConsensusData.Get().(*ClientMeta_AdditionalNodeRecordConsensusData) +} + +var vtprotoPool_ClientMeta_AdditionalConsensusEngineAPINewPayloadData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalConsensusEngineAPINewPayloadData{} + }, +} + +func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalConsensusEngineAPINewPayloadData.Put(m) + } +} +func ClientMeta_AdditionalConsensusEngineAPINewPayloadDataFromVTPool() *ClientMeta_AdditionalConsensusEngineAPINewPayloadData { + return vtprotoPool_ClientMeta_AdditionalConsensusEngineAPINewPayloadData.Get().(*ClientMeta_AdditionalConsensusEngineAPINewPayloadData) +} + +var vtprotoPool_ClientMeta_AdditionalConsensusEngineAPIGetBlobsData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalConsensusEngineAPIGetBlobsData{} + }, +} + +func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalConsensusEngineAPIGetBlobsData.Put(m) + } +} +func ClientMeta_AdditionalConsensusEngineAPIGetBlobsDataFromVTPool() *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData { + return vtprotoPool_ClientMeta_AdditionalConsensusEngineAPIGetBlobsData.Get().(*ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) +} + +var vtprotoPool_ClientMeta_AdditionalEthV1BeaconBlobData = sync.Pool{ + New: func() interface{} { + return &ClientMeta_AdditionalEthV1BeaconBlobData{} + }, +} + +func (m *ClientMeta_AdditionalEthV1BeaconBlobData) ResetVT() { + if m != nil { + m.Epoch.ReturnToVTPool() + m.Slot.ReturnToVTPool() + m.Reset() + } +} +func (m *ClientMeta_AdditionalEthV1BeaconBlobData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta_AdditionalEthV1BeaconBlobData.Put(m) + } +} +func ClientMeta_AdditionalEthV1BeaconBlobDataFromVTPool() *ClientMeta_AdditionalEthV1BeaconBlobData { + return vtprotoPool_ClientMeta_AdditionalEthV1BeaconBlobData.Get().(*ClientMeta_AdditionalEthV1BeaconBlobData) +} + +var vtprotoPool_ClientMeta = sync.Pool{ + New: func() interface{} { + return &ClientMeta{} + }, +} + +func (m *ClientMeta) ResetVT() { + if m != nil { + m.Ethereum.ReturnToVTPool() + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsAttestation); ok { + oneof.EthV1EventsAttestation.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsHead); ok { + oneof.EthV1EventsHead.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsBlock); ok { + oneof.EthV1EventsBlock.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsVoluntaryExit); ok { + oneof.EthV1EventsVoluntaryExit.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsFinalizedCheckpoint); ok { + oneof.EthV1EventsFinalizedCheckpoint.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsChainReorg); ok { + oneof.EthV1EventsChainReorg.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsContributionAndProof); ok { + oneof.EthV1EventsContributionAndProof.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MempoolTransaction); ok { + oneof.MempoolTransaction.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlock); ok { + oneof.EthV2BeaconBlock.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1DebugForkChoice); ok { + oneof.EthV1DebugForkChoice.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1DebugForkChoiceReorg); ok { + oneof.EthV1DebugForkChoiceReorg.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1BeaconCommittee); ok { + oneof.EthV1BeaconCommittee.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1ValidatorAttestationData); ok { + oneof.EthV1ValidatorAttestationData.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsAttestationV2); ok { + oneof.EthV1EventsAttestationV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsHeadV2); ok { + oneof.EthV1EventsHeadV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsBlockV2); ok { + oneof.EthV1EventsBlockV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsVoluntaryExitV2); ok { + oneof.EthV1EventsVoluntaryExitV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsFinalizedCheckpointV2); ok { + oneof.EthV1EventsFinalizedCheckpointV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsChainReorgV2); ok { + oneof.EthV1EventsChainReorgV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsContributionAndProofV2); ok { + oneof.EthV1EventsContributionAndProofV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MempoolTransactionV2); ok { + oneof.MempoolTransactionV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockV2); ok { + oneof.EthV2BeaconBlockV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1DebugForkChoiceV2); ok { + oneof.EthV1DebugForkChoiceV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1DebugForkChoiceReorgV2); ok { + oneof.EthV1DebugForkChoiceReorgV2.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockAttesterSlashing); ok { + oneof.EthV2BeaconBlockAttesterSlashing.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockProposerSlashing); ok { + oneof.EthV2BeaconBlockProposerSlashing.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockVoluntaryExit); ok { + oneof.EthV2BeaconBlockVoluntaryExit.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockDeposit); ok { + oneof.EthV2BeaconBlockDeposit.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockBlsToExecutionChange); ok { + oneof.EthV2BeaconBlockBlsToExecutionChange.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockExecutionTransaction); ok { + oneof.EthV2BeaconBlockExecutionTransaction.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockWithdrawal); ok { + oneof.EthV2BeaconBlockWithdrawal.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsBlobSidecar); ok { + oneof.EthV1EventsBlobSidecar.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1BeaconBlobSidecar); ok { + oneof.EthV1BeaconBlobSidecar.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_BeaconP2PAttestation); ok { + oneof.BeaconP2PAttestation.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1ProposerDuty); ok { + oneof.EthV1ProposerDuty.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockElaboratedAttestation); ok { + oneof.EthV2BeaconBlockElaboratedAttestation.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceAddPeer); ok { + oneof.Libp2PTraceAddPeer.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRemovePeer); ok { + oneof.Libp2PTraceRemovePeer.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRecvRpc); ok { + oneof.Libp2PTraceRecvRpc.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceSendRpc); ok { + oneof.Libp2PTraceSendRpc.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceJoin); ok { + oneof.Libp2PTraceJoin.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceConnected); ok { + oneof.Libp2PTraceConnected.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceDisconnected); ok { + oneof.Libp2PTraceDisconnected.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceHandleMetadata); ok { + oneof.Libp2PTraceHandleMetadata.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceHandleStatus); ok { + oneof.Libp2PTraceHandleStatus.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubBeaconBlock); ok { + oneof.Libp2PTraceGossipsubBeaconBlock.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubBeaconAttestation); ok { + oneof.Libp2PTraceGossipsubBeaconAttestation.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubBlobSidecar); ok { + oneof.Libp2PTraceGossipsubBlobSidecar.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1Validators); ok { + oneof.EthV1Validators.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MevRelayBidTraceBuilderBlockSubmission); ok { + oneof.MevRelayBidTraceBuilderBlockSubmission.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MevRelayPayloadDelivered); ok { + oneof.MevRelayPayloadDelivered.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV3ValidatorBlock); ok { + oneof.EthV3ValidatorBlock.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_MevRelayValidatorRegistration); ok { + oneof.MevRelayValidatorRegistration.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsBlockGossip); ok { + oneof.EthV1EventsBlockGossip.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceDropRpc); ok { + oneof.Libp2PTraceDropRpc.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceLeave); ok { + oneof.Libp2PTraceLeave.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGraft); ok { + oneof.Libp2PTraceGraft.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTracePrune); ok { + oneof.Libp2PTracePrune.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceDuplicateMessage); ok { + oneof.Libp2PTraceDuplicateMessage.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceDeliverMessage); ok { + oneof.Libp2PTraceDeliverMessage.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTracePublishMessage); ok { + oneof.Libp2PTracePublishMessage.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRejectMessage); ok { + oneof.Libp2PTraceRejectMessage.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlIhave); ok { + oneof.Libp2PTraceRpcMetaControlIhave.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlIwant); ok { + oneof.Libp2PTraceRpcMetaControlIwant.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlIdontwant); ok { + oneof.Libp2PTraceRpcMetaControlIdontwant.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlGraft); ok { + oneof.Libp2PTraceRpcMetaControlGraft.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaControlPrune); ok { + oneof.Libp2PTraceRpcMetaControlPrune.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaSubscription); ok { + oneof.Libp2PTraceRpcMetaSubscription.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcMetaMessage); ok { + oneof.Libp2PTraceRpcMetaMessage.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_NodeRecordConsensus); ok { + oneof.NodeRecordConsensus.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubAggregateAndProof); ok { + oneof.Libp2PTraceGossipsubAggregateAndProof.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1EventsDataColumnSidecar); ok { + oneof.EthV1EventsDataColumnSidecar.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceGossipsubDataColumnSidecar); ok { + oneof.Libp2PTraceGossipsubDataColumnSidecar.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceSyntheticHeartbeat); ok { + oneof.Libp2PTraceSyntheticHeartbeat.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceRpcDataColumnCustodyProbe); ok { + oneof.Libp2PTraceRpcDataColumnCustodyProbe.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_ConsensusEngineApiNewPayload); ok { + oneof.ConsensusEngineApiNewPayload.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_ConsensusEngineApiGetBlobs); ok { + oneof.ConsensusEngineApiGetBlobs.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1BeaconBlob); ok { + oneof.EthV1BeaconBlob.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV1BeaconSyncCommittee); ok { + oneof.EthV1BeaconSyncCommittee.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_EthV2BeaconBlockSyncAggregate); ok { + oneof.EthV2BeaconBlockSyncAggregate.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ClientMeta_Libp2PTraceIdentify); ok { + oneof.Libp2PTraceIdentify.ReturnToVTPool() + } + m.Reset() + } +} +func (m *ClientMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ClientMeta.Put(m) + } +} +func ClientMetaFromVTPool() *ClientMeta { + return vtprotoPool_ClientMeta.Get().(*ClientMeta) +} + +var vtprotoPool_ServerMeta_Event = sync.Pool{ + New: func() interface{} { + return &ServerMeta_Event{} + }, +} + +func (m *ServerMeta_Event) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ServerMeta_Event) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_Event.Put(m) + } +} +func ServerMeta_EventFromVTPool() *ServerMeta_Event { + return vtprotoPool_ServerMeta_Event.Get().(*ServerMeta_Event) +} + +var vtprotoPool_ServerMeta_Geo = sync.Pool{ + New: func() interface{} { + return &ServerMeta_Geo{} + }, +} + +func (m *ServerMeta_Geo) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ServerMeta_Geo) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_Geo.Put(m) + } +} +func ServerMeta_GeoFromVTPool() *ServerMeta_Geo { + return vtprotoPool_ServerMeta_Geo.Get().(*ServerMeta_Geo) +} + +var vtprotoPool_ServerMeta_Client = sync.Pool{ + New: func() interface{} { + return &ServerMeta_Client{} + }, +} + +func (m *ServerMeta_Client) ResetVT() { + if m != nil { + m.Geo.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_Client) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_Client.Put(m) + } +} +func ServerMeta_ClientFromVTPool() *ServerMeta_Client { + return vtprotoPool_ServerMeta_Client.Get().(*ServerMeta_Client) +} + +var vtprotoPool_ServerMeta_Peer = sync.Pool{ + New: func() interface{} { + return &ServerMeta_Peer{} + }, +} + +func (m *ServerMeta_Peer) ResetVT() { + if m != nil { + m.Geo.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_Peer) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_Peer.Put(m) + } +} +func ServerMeta_PeerFromVTPool() *ServerMeta_Peer { + return vtprotoPool_ServerMeta_Peer.Get().(*ServerMeta_Peer) +} + +var vtprotoPool_ServerMeta_AdditionalBeaconP2PAttestationData = sync.Pool{ + New: func() interface{} { + return &ServerMeta_AdditionalBeaconP2PAttestationData{} + }, +} + +func (m *ServerMeta_AdditionalBeaconP2PAttestationData) ResetVT() { + if m != nil { + m.Peer.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_AdditionalBeaconP2PAttestationData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_AdditionalBeaconP2PAttestationData.Put(m) + } +} +func ServerMeta_AdditionalBeaconP2PAttestationDataFromVTPool() *ServerMeta_AdditionalBeaconP2PAttestationData { + return vtprotoPool_ServerMeta_AdditionalBeaconP2PAttestationData.Get().(*ServerMeta_AdditionalBeaconP2PAttestationData) +} + +var vtprotoPool_ServerMeta_AdditionalLibp2PTraceConnectedData = sync.Pool{ + New: func() interface{} { + return &ServerMeta_AdditionalLibp2PTraceConnectedData{} + }, +} + +func (m *ServerMeta_AdditionalLibp2PTraceConnectedData) ResetVT() { + if m != nil { + m.Peer.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_AdditionalLibp2PTraceConnectedData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_AdditionalLibp2PTraceConnectedData.Put(m) + } +} +func ServerMeta_AdditionalLibp2PTraceConnectedDataFromVTPool() *ServerMeta_AdditionalLibp2PTraceConnectedData { + return vtprotoPool_ServerMeta_AdditionalLibp2PTraceConnectedData.Get().(*ServerMeta_AdditionalLibp2PTraceConnectedData) +} + +var vtprotoPool_ServerMeta_AdditionalLibp2PTraceDisconnectedData = sync.Pool{ + New: func() interface{} { + return &ServerMeta_AdditionalLibp2PTraceDisconnectedData{} + }, +} + +func (m *ServerMeta_AdditionalLibp2PTraceDisconnectedData) ResetVT() { + if m != nil { + m.Peer.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_AdditionalLibp2PTraceDisconnectedData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_AdditionalLibp2PTraceDisconnectedData.Put(m) + } +} +func ServerMeta_AdditionalLibp2PTraceDisconnectedDataFromVTPool() *ServerMeta_AdditionalLibp2PTraceDisconnectedData { + return vtprotoPool_ServerMeta_AdditionalLibp2PTraceDisconnectedData.Get().(*ServerMeta_AdditionalLibp2PTraceDisconnectedData) +} + +var vtprotoPool_ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData = sync.Pool{ + New: func() interface{} { + return &ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData{} + }, +} + +func (m *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) ResetVT() { + if m != nil { + m.Peer.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData.Put(m) + } +} +func ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatDataFromVTPool() *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData { + return vtprotoPool_ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData.Get().(*ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) +} + +var vtprotoPool_ServerMeta_AdditionalLibp2PTraceIdentifyData = sync.Pool{ + New: func() interface{} { + return &ServerMeta_AdditionalLibp2PTraceIdentifyData{} + }, +} + +func (m *ServerMeta_AdditionalLibp2PTraceIdentifyData) ResetVT() { + if m != nil { + m.Peer.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_AdditionalLibp2PTraceIdentifyData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_AdditionalLibp2PTraceIdentifyData.Put(m) + } +} +func ServerMeta_AdditionalLibp2PTraceIdentifyDataFromVTPool() *ServerMeta_AdditionalLibp2PTraceIdentifyData { + return vtprotoPool_ServerMeta_AdditionalLibp2PTraceIdentifyData.Get().(*ServerMeta_AdditionalLibp2PTraceIdentifyData) +} + +var vtprotoPool_ServerMeta_AdditionalNodeRecordConsensusData = sync.Pool{ + New: func() interface{} { + return &ServerMeta_AdditionalNodeRecordConsensusData{} + }, +} + +func (m *ServerMeta_AdditionalNodeRecordConsensusData) ResetVT() { + if m != nil { + m.Geo.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_AdditionalNodeRecordConsensusData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_AdditionalNodeRecordConsensusData.Put(m) + } +} +func ServerMeta_AdditionalNodeRecordConsensusDataFromVTPool() *ServerMeta_AdditionalNodeRecordConsensusData { + return vtprotoPool_ServerMeta_AdditionalNodeRecordConsensusData.Get().(*ServerMeta_AdditionalNodeRecordConsensusData) +} + +var vtprotoPool_ServerMeta_AdditionalNodeRecordExecutionData = sync.Pool{ + New: func() interface{} { + return &ServerMeta_AdditionalNodeRecordExecutionData{} + }, +} + +func (m *ServerMeta_AdditionalNodeRecordExecutionData) ResetVT() { + if m != nil { + m.Geo.ReturnToVTPool() + m.Reset() + } +} +func (m *ServerMeta_AdditionalNodeRecordExecutionData) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta_AdditionalNodeRecordExecutionData.Put(m) + } +} +func ServerMeta_AdditionalNodeRecordExecutionDataFromVTPool() *ServerMeta_AdditionalNodeRecordExecutionData { + return vtprotoPool_ServerMeta_AdditionalNodeRecordExecutionData.Get().(*ServerMeta_AdditionalNodeRecordExecutionData) +} + +var vtprotoPool_ServerMeta = sync.Pool{ + New: func() interface{} { + return &ServerMeta{} + }, +} + +func (m *ServerMeta) ResetVT() { + if m != nil { + m.Event.ReturnToVTPool() + m.Client.ReturnToVTPool() + if oneof, ok := m.AdditionalData.(*ServerMeta_BEACON_P2P_ATTESTATION); ok { + oneof.BEACON_P2P_ATTESTATION.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ServerMeta_LIBP2P_TRACE_CONNECTED); ok { + oneof.LIBP2P_TRACE_CONNECTED.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ServerMeta_LIBP2P_TRACE_DISCONNECTED); ok { + oneof.LIBP2P_TRACE_DISCONNECTED.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ServerMeta_NODE_RECORD_CONSENSUS); ok { + oneof.NODE_RECORD_CONSENSUS.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ServerMeta_NODE_RECORD_EXECUTION); ok { + oneof.NODE_RECORD_EXECUTION.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ServerMeta_LIBP2P_TRACE_SYNTHETIC_HEARTBEAT); ok { + oneof.LIBP2P_TRACE_SYNTHETIC_HEARTBEAT.ReturnToVTPool() + } + if oneof, ok := m.AdditionalData.(*ServerMeta_LIBP2P_TRACE_IDENTIFY); ok { + oneof.LIBP2P_TRACE_IDENTIFY.ReturnToVTPool() + } + m.Reset() + } +} +func (m *ServerMeta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ServerMeta.Put(m) + } +} +func ServerMetaFromVTPool() *ServerMeta { + return vtprotoPool_ServerMeta.Get().(*ServerMeta) +} + +var vtprotoPool_Meta = sync.Pool{ + New: func() interface{} { + return &Meta{} + }, +} + +func (m *Meta) ResetVT() { + if m != nil { + m.Client.ReturnToVTPool() + m.Server.ReturnToVTPool() + m.Reset() + } +} +func (m *Meta) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Meta.Put(m) + } +} +func MetaFromVTPool() *Meta { + return vtprotoPool_Meta.Get().(*Meta) +} + +var vtprotoPool_Event = sync.Pool{ + New: func() interface{} { + return &Event{} + }, +} + +func (m *Event) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *Event) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_Event.Put(m) + } +} +func EventFromVTPool() *Event { + return vtprotoPool_Event.Get().(*Event) +} + +var vtprotoPool_ExecutionBlockMetrics_StateReads = sync.Pool{ + New: func() interface{} { + return &ExecutionBlockMetrics_StateReads{} + }, +} + +func (m *ExecutionBlockMetrics_StateReads) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionBlockMetrics_StateReads) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionBlockMetrics_StateReads.Put(m) + } +} +func ExecutionBlockMetrics_StateReadsFromVTPool() *ExecutionBlockMetrics_StateReads { + return vtprotoPool_ExecutionBlockMetrics_StateReads.Get().(*ExecutionBlockMetrics_StateReads) +} + +var vtprotoPool_ExecutionBlockMetrics_StateWrites = sync.Pool{ + New: func() interface{} { + return &ExecutionBlockMetrics_StateWrites{} + }, +} + +func (m *ExecutionBlockMetrics_StateWrites) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionBlockMetrics_StateWrites) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionBlockMetrics_StateWrites.Put(m) + } +} +func ExecutionBlockMetrics_StateWritesFromVTPool() *ExecutionBlockMetrics_StateWrites { + return vtprotoPool_ExecutionBlockMetrics_StateWrites.Get().(*ExecutionBlockMetrics_StateWrites) +} + +var vtprotoPool_ExecutionBlockMetrics_CacheEntry = sync.Pool{ + New: func() interface{} { + return &ExecutionBlockMetrics_CacheEntry{} + }, +} + +func (m *ExecutionBlockMetrics_CacheEntry) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionBlockMetrics_CacheEntry) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionBlockMetrics_CacheEntry.Put(m) + } +} +func ExecutionBlockMetrics_CacheEntryFromVTPool() *ExecutionBlockMetrics_CacheEntry { + return vtprotoPool_ExecutionBlockMetrics_CacheEntry.Get().(*ExecutionBlockMetrics_CacheEntry) +} + +var vtprotoPool_ExecutionBlockMetrics_CodeCacheEntry = sync.Pool{ + New: func() interface{} { + return &ExecutionBlockMetrics_CodeCacheEntry{} + }, +} + +func (m *ExecutionBlockMetrics_CodeCacheEntry) ResetVT() { + if m != nil { + m.Reset() + } +} +func (m *ExecutionBlockMetrics_CodeCacheEntry) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionBlockMetrics_CodeCacheEntry.Put(m) + } +} +func ExecutionBlockMetrics_CodeCacheEntryFromVTPool() *ExecutionBlockMetrics_CodeCacheEntry { + return vtprotoPool_ExecutionBlockMetrics_CodeCacheEntry.Get().(*ExecutionBlockMetrics_CodeCacheEntry) +} + +var vtprotoPool_ExecutionBlockMetrics = sync.Pool{ + New: func() interface{} { + return &ExecutionBlockMetrics{} + }, +} + +func (m *ExecutionBlockMetrics) ResetVT() { + if m != nil { + m.StateReads.ReturnToVTPool() + m.StateWrites.ReturnToVTPool() + m.AccountCache.ReturnToVTPool() + m.StorageCache.ReturnToVTPool() + m.CodeCache.ReturnToVTPool() + m.Reset() + } +} +func (m *ExecutionBlockMetrics) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_ExecutionBlockMetrics.Put(m) + } +} +func ExecutionBlockMetricsFromVTPool() *ExecutionBlockMetrics { + return vtprotoPool_ExecutionBlockMetrics.Get().(*ExecutionBlockMetrics) +} + +var vtprotoPool_DecoratedEvent = sync.Pool{ + New: func() interface{} { + return &DecoratedEvent{} + }, +} + +func (m *DecoratedEvent) ResetVT() { + if m != nil { + m.Event.ReturnToVTPool() + m.Meta.ReturnToVTPool() + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsAttestation); ok { + oneof.EthV1EventsAttestation.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsBlock); ok { + oneof.EthV1EventsBlock.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsChainReorg); ok { + oneof.EthV1EventsChainReorg.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsFinalizedCheckpoint); ok { + oneof.EthV1EventsFinalizedCheckpoint.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsHead); ok { + oneof.EthV1EventsHead.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsVoluntaryExit); ok { + oneof.EthV1EventsVoluntaryExit.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsContributionAndProof); ok { + oneof.EthV1EventsContributionAndProof.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlock); ok { + oneof.EthV2BeaconBlock.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ForkChoice); ok { + oneof.EthV1ForkChoice.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ForkChoiceReorg); ok { + oneof.EthV1ForkChoiceReorg.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1BeaconCommittee); ok { + oneof.EthV1BeaconCommittee.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ValidatorAttestationData); ok { + oneof.EthV1ValidatorAttestationData.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsAttestationV2); ok { + oneof.EthV1EventsAttestationV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsBlockV2); ok { + oneof.EthV1EventsBlockV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsChainReorgV2); ok { + oneof.EthV1EventsChainReorgV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsFinalizedCheckpointV2); ok { + oneof.EthV1EventsFinalizedCheckpointV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsHeadV2); ok { + oneof.EthV1EventsHeadV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsVoluntaryExitV2); ok { + oneof.EthV1EventsVoluntaryExitV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsContributionAndProofV2); ok { + oneof.EthV1EventsContributionAndProofV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockV2); ok { + oneof.EthV2BeaconBlockV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ForkChoiceV2); ok { + oneof.EthV1ForkChoiceV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ForkChoiceReorgV2); ok { + oneof.EthV1ForkChoiceReorgV2.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockAttesterSlashing); ok { + oneof.EthV2BeaconBlockAttesterSlashing.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockProposerSlashing); ok { + oneof.EthV2BeaconBlockProposerSlashing.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockVoluntaryExit); ok { + oneof.EthV2BeaconBlockVoluntaryExit.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockDeposit); ok { + oneof.EthV2BeaconBlockDeposit.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockBlsToExecutionChange); ok { + oneof.EthV2BeaconBlockBlsToExecutionChange.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockExecutionTransaction); ok { + oneof.EthV2BeaconBlockExecutionTransaction.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockWithdrawal); ok { + oneof.EthV2BeaconBlockWithdrawal.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsBlobSidecar); ok { + oneof.EthV1EventsBlobSidecar.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1BeaconBlockBlobSidecar); ok { + oneof.EthV1BeaconBlockBlobSidecar.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_BeaconP2PAttestation); ok { + oneof.BeaconP2PAttestation.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1ProposerDuty); ok { + oneof.EthV1ProposerDuty.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockElaboratedAttestation); ok { + oneof.EthV2BeaconBlockElaboratedAttestation.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceAddPeer); ok { + oneof.Libp2PTraceAddPeer.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRemovePeer); ok { + oneof.Libp2PTraceRemovePeer.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRecvRpc); ok { + oneof.Libp2PTraceRecvRpc.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceSendRpc); ok { + oneof.Libp2PTraceSendRpc.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceJoin); ok { + oneof.Libp2PTraceJoin.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceConnected); ok { + oneof.Libp2PTraceConnected.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceDisconnected); ok { + oneof.Libp2PTraceDisconnected.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceHandleMetadata); ok { + oneof.Libp2PTraceHandleMetadata.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceHandleStatus); ok { + oneof.Libp2PTraceHandleStatus.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubBeaconBlock); ok { + oneof.Libp2PTraceGossipsubBeaconBlock.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation); ok { + oneof.Libp2PTraceGossipsubBeaconAttestation.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubBlobSidecar); ok { + oneof.Libp2PTraceGossipsubBlobSidecar.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1Validators); ok { + oneof.EthV1Validators.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_MevRelayBidTraceBuilderBlockSubmission); ok { + oneof.MevRelayBidTraceBuilderBlockSubmission.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_MevRelayPayloadDelivered); ok { + oneof.MevRelayPayloadDelivered.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV3ValidatorBlock); ok { + oneof.EthV3ValidatorBlock.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_MevRelayValidatorRegistration); ok { + oneof.MevRelayValidatorRegistration.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsBlockGossip); ok { + oneof.EthV1EventsBlockGossip.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceDropRpc); ok { + oneof.Libp2PTraceDropRpc.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceLeave); ok { + oneof.Libp2PTraceLeave.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGraft); ok { + oneof.Libp2PTraceGraft.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTracePrune); ok { + oneof.Libp2PTracePrune.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceDuplicateMessage); ok { + oneof.Libp2PTraceDuplicateMessage.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceDeliverMessage); ok { + oneof.Libp2PTraceDeliverMessage.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTracePublishMessage); ok { + oneof.Libp2PTracePublishMessage.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRejectMessage); ok { + oneof.Libp2PTraceRejectMessage.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlIhave); ok { + oneof.Libp2PTraceRpcMetaControlIhave.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlIwant); ok { + oneof.Libp2PTraceRpcMetaControlIwant.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlIdontwant); ok { + oneof.Libp2PTraceRpcMetaControlIdontwant.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlGraft); ok { + oneof.Libp2PTraceRpcMetaControlGraft.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaControlPrune); ok { + oneof.Libp2PTraceRpcMetaControlPrune.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaSubscription); ok { + oneof.Libp2PTraceRpcMetaSubscription.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcMetaMessage); ok { + oneof.Libp2PTraceRpcMetaMessage.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_NodeRecordConsensus); ok { + oneof.NodeRecordConsensus.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_NodeRecordExecution); ok { + oneof.NodeRecordExecution.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubAggregateAndProof); ok { + oneof.Libp2PTraceGossipsubAggregateAndProof.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1EventsDataColumnSidecar); ok { + oneof.EthV1EventsDataColumnSidecar.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceGossipsubDataColumnSidecar); ok { + oneof.Libp2PTraceGossipsubDataColumnSidecar.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceSyntheticHeartbeat); ok { + oneof.Libp2PTraceSyntheticHeartbeat.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceIdentify); ok { + oneof.Libp2PTraceIdentify.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_Libp2PTraceRpcDataColumnCustodyProbe); ok { + oneof.Libp2PTraceRpcDataColumnCustodyProbe.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_ExecutionStateSize); ok { + oneof.ExecutionStateSize.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_ConsensusEngineApiNewPayload); ok { + oneof.ConsensusEngineApiNewPayload.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_ConsensusEngineApiGetBlobs); ok { + oneof.ConsensusEngineApiGetBlobs.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_ExecutionEngineNewPayload); ok { + oneof.ExecutionEngineNewPayload.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_ExecutionEngineGetBlobs); ok { + oneof.ExecutionEngineGetBlobs.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1BeaconBlob); ok { + oneof.EthV1BeaconBlob.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV1BeaconSyncCommittee); ok { + oneof.EthV1BeaconSyncCommittee.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_EthV2BeaconBlockSyncAggregate); ok { + oneof.EthV2BeaconBlockSyncAggregate.ReturnToVTPool() + } + if oneof, ok := m.Data.(*DecoratedEvent_ExecutionBlockMetrics); ok { + oneof.ExecutionBlockMetrics.ReturnToVTPool() + } + m.Reset() + } +} +func (m *DecoratedEvent) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_DecoratedEvent.Put(m) + } +} +func DecoratedEventFromVTPool() *DecoratedEvent { + return vtprotoPool_DecoratedEvent.Get().(*DecoratedEvent) +} func (m *CreateEventsRequest) SizeVT() (n int) { if m == nil { return 0 @@ -18068,7 +21765,14 @@ func (m *CreateEventsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Events = append(m.Events, &DecoratedEvent{}) + if len(m.Events) == cap(m.Events) { + m.Events = append(m.Events, &DecoratedEvent{}) + } else { + m.Events = m.Events[:len(m.Events)+1] + if m.Events[len(m.Events)-1] == nil { + m.Events[len(m.Events)-1] = &DecoratedEvent{} + } + } if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -19183,7 +22887,7 @@ func (m *DebugForkChoiceReorg) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Before == nil { - m.Before = &v1.ForkChoice{} + m.Before = v1.ForkChoiceFromVTPool() } if err := m.Before.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -19219,7 +22923,7 @@ func (m *DebugForkChoiceReorg) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.After == nil { - m.After = &v1.ForkChoice{} + m.After = v1.ForkChoiceFromVTPool() } if err := m.After.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -19255,7 +22959,7 @@ func (m *DebugForkChoiceReorg) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Event == nil { - m.Event = &v1.EventChainReorg{} + m.Event = v1.EventChainReorgFromVTPool() } if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -19342,7 +23046,7 @@ func (m *DebugForkChoiceReorgV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Before == nil { - m.Before = &v1.ForkChoiceV2{} + m.Before = v1.ForkChoiceV2FromVTPool() } if err := m.Before.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -19378,7 +23082,7 @@ func (m *DebugForkChoiceReorgV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.After == nil { - m.After = &v1.ForkChoiceV2{} + m.After = v1.ForkChoiceV2FromVTPool() } if err := m.After.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -19414,7 +23118,7 @@ func (m *DebugForkChoiceReorgV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Event == nil { - m.Event = &v1.EventChainReorgV2{} + m.Event = v1.EventChainReorgV2FromVTPool() } if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -19500,7 +23204,14 @@ func (m *Validators) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Validators = append(m.Validators, &v1.Validator{}) + if len(m.Validators) == cap(m.Validators) { + m.Validators = append(m.Validators, &v1.Validator{}) + } else { + m.Validators = m.Validators[:len(m.Validators)+1] + if m.Validators[len(m.Validators)-1] == nil { + m.Validators[len(m.Validators)-1] = &v1.Validator{} + } + } if err := m.Validators[len(m.Validators)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -19586,7 +23297,7 @@ func (m *SyncCommitteeData) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.SyncCommittee == nil { - m.SyncCommittee = &v1.SyncCommittee{} + m.SyncCommittee = v1.SyncCommitteeFromVTPool() } if err := m.SyncCommittee.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -19736,7 +23447,14 @@ func (m *SyncAggregateData) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ValidatorsParticipated = append(m.ValidatorsParticipated, &wrapperspb1.UInt64Value{}) + if len(m.ValidatorsParticipated) == cap(m.ValidatorsParticipated) { + m.ValidatorsParticipated = append(m.ValidatorsParticipated, &wrapperspb1.UInt64Value{}) + } else { + m.ValidatorsParticipated = m.ValidatorsParticipated[:len(m.ValidatorsParticipated)+1] + if m.ValidatorsParticipated[len(m.ValidatorsParticipated)-1] == nil { + m.ValidatorsParticipated[len(m.ValidatorsParticipated)-1] = &wrapperspb1.UInt64Value{} + } + } if err := (*wrapperspb.UInt64Value)(m.ValidatorsParticipated[len(m.ValidatorsParticipated)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -19770,7 +23488,14 @@ func (m *SyncAggregateData) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ValidatorsMissed = append(m.ValidatorsMissed, &wrapperspb1.UInt64Value{}) + if len(m.ValidatorsMissed) == cap(m.ValidatorsMissed) { + m.ValidatorsMissed = append(m.ValidatorsMissed, &wrapperspb1.UInt64Value{}) + } else { + m.ValidatorsMissed = m.ValidatorsMissed[:len(m.ValidatorsMissed)+1] + if m.ValidatorsMissed[len(m.ValidatorsMissed)-1] == nil { + m.ValidatorsMissed[len(m.ValidatorsMissed)-1] = &wrapperspb1.UInt64Value{} + } + } if err := (*wrapperspb.UInt64Value)(m.ValidatorsMissed[len(m.ValidatorsMissed)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -19892,7 +23617,7 @@ func (m *BlockIdentifier) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -19928,7 +23653,7 @@ func (m *BlockIdentifier) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -22372,7 +26097,14 @@ func (m *ExecutionEngineGetBlobs) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ReturnedBlobIndexes = append(m.ReturnedBlobIndexes, &wrapperspb1.UInt32Value{}) + if len(m.ReturnedBlobIndexes) == cap(m.ReturnedBlobIndexes) { + m.ReturnedBlobIndexes = append(m.ReturnedBlobIndexes, &wrapperspb1.UInt32Value{}) + } else { + m.ReturnedBlobIndexes = m.ReturnedBlobIndexes[:len(m.ReturnedBlobIndexes)+1] + if m.ReturnedBlobIndexes[len(m.ReturnedBlobIndexes)-1] == nil { + m.ReturnedBlobIndexes[len(m.ReturnedBlobIndexes)-1] = &wrapperspb1.UInt32Value{} + } + } if err := (*wrapperspb.UInt32Value)(m.ReturnedBlobIndexes[len(m.ReturnedBlobIndexes)-1]).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -22560,7 +26292,7 @@ func (m *ClientMeta_Ethereum_Execution) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ForkId == nil { - m.ForkId = &ForkID{} + m.ForkId = ForkIDFromVTPool() } if err := m.ForkId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -22922,7 +26654,7 @@ func (m *ClientMeta_Ethereum) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Network == nil { - m.Network = &ClientMeta_Ethereum_Network{} + m.Network = ClientMeta_Ethereum_NetworkFromVTPool() } if err := m.Network.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -22958,7 +26690,7 @@ func (m *ClientMeta_Ethereum) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Execution == nil { - m.Execution = &ClientMeta_Ethereum_Execution{} + m.Execution = ClientMeta_Ethereum_ExecutionFromVTPool() } if err := m.Execution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -22994,7 +26726,7 @@ func (m *ClientMeta_Ethereum) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Consensus == nil { - m.Consensus = &ClientMeta_Ethereum_Consensus{} + m.Consensus = ClientMeta_Ethereum_ConsensusFromVTPool() } if err := m.Consensus.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23081,7 +26813,7 @@ func (m *ClientMeta_AdditionalEthV1AttestationSourceData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23168,7 +26900,7 @@ func (m *ClientMeta_AdditionalEthV1AttestationSourceV2Data) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23255,7 +26987,7 @@ func (m *ClientMeta_AdditionalEthV1AttestationTargetData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23342,7 +27074,7 @@ func (m *ClientMeta_AdditionalEthV1AttestationTargetV2Data) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23429,7 +27161,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Source == nil { - m.Source = &ClientMeta_AdditionalEthV1AttestationSourceData{} + m.Source = ClientMeta_AdditionalEthV1AttestationSourceDataFromVTPool() } if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23465,7 +27197,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Target == nil { - m.Target = &ClientMeta_AdditionalEthV1AttestationTargetData{} + m.Target = ClientMeta_AdditionalEthV1AttestationTargetDataFromVTPool() } if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23501,7 +27233,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &Slot{} + m.Slot = SlotFromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23537,7 +27269,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23573,7 +27305,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &Propagation{} + m.Propagation = PropagationFromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23609,7 +27341,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.AttestingValidator == nil { - m.AttestingValidator = &AttestingValidator{} + m.AttestingValidator = AttestingValidatorFromVTPool() } if err := m.AttestingValidator.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23696,7 +27428,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Source == nil { - m.Source = &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + m.Source = ClientMeta_AdditionalEthV1AttestationSourceV2DataFromVTPool() } if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23732,7 +27464,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Target == nil { - m.Target = &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + m.Target = ClientMeta_AdditionalEthV1AttestationTargetV2DataFromVTPool() } if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23768,7 +27500,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23804,7 +27536,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23840,7 +27572,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23876,7 +27608,7 @@ func (m *ClientMeta_AdditionalEthV1EventsAttestationV2Data) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.AttestingValidator == nil { - m.AttestingValidator = &AttestingValidatorV2{} + m.AttestingValidator = AttestingValidatorV2FromVTPool() } if err := m.AttestingValidator.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23963,7 +27695,7 @@ func (m *ClientMeta_AdditionalEthV1EventsHeadData) UnmarshalVT(dAtA []byte) erro return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -23999,7 +27731,7 @@ func (m *ClientMeta_AdditionalEthV1EventsHeadData) UnmarshalVT(dAtA []byte) erro return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &Slot{} + m.Slot = SlotFromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24035,7 +27767,7 @@ func (m *ClientMeta_AdditionalEthV1EventsHeadData) UnmarshalVT(dAtA []byte) erro return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &Propagation{} + m.Propagation = PropagationFromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24122,7 +27854,7 @@ func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) UnmarshalVT(dAtA []byte) er return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24158,7 +27890,7 @@ func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) UnmarshalVT(dAtA []byte) er return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24194,7 +27926,7 @@ func (m *ClientMeta_AdditionalEthV1EventsHeadV2Data) UnmarshalVT(dAtA []byte) er return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24281,7 +28013,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockData) UnmarshalVT(dAtA []byte) err return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24317,7 +28049,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockData) UnmarshalVT(dAtA []byte) err return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &Slot{} + m.Slot = SlotFromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24353,7 +28085,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockData) UnmarshalVT(dAtA []byte) err return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &Propagation{} + m.Propagation = PropagationFromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24440,7 +28172,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24476,7 +28208,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24512,7 +28244,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockV2Data) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24599,7 +28331,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24635,7 +28367,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24671,7 +28403,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlockGossipData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24758,7 +28490,7 @@ func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24845,7 +28577,7 @@ func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) UnmarshalVT(dAtA [ return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24881,7 +28613,7 @@ func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) UnmarshalVT(dAtA [ return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -24917,7 +28649,7 @@ func (m *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) UnmarshalVT(dAtA [ return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25004,7 +28736,7 @@ func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) UnmarshalVT(dA return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25091,7 +28823,7 @@ func (m *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) UnmarshalVT( return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25178,7 +28910,7 @@ func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) UnmarshalVT(dAtA []byte return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25214,7 +28946,7 @@ func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) UnmarshalVT(dAtA []byte return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &Slot{} + m.Slot = SlotFromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25250,7 +28982,7 @@ func (m *ClientMeta_AdditionalEthV1EventsChainReorgData) UnmarshalVT(dAtA []byte return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &Propagation{} + m.Propagation = PropagationFromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25337,7 +29069,7 @@ func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) UnmarshalVT(dAtA []by return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25373,7 +29105,7 @@ func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) UnmarshalVT(dAtA []by return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25409,7 +29141,7 @@ func (m *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) UnmarshalVT(dAtA []by return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25496,7 +29228,7 @@ func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) U return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25532,7 +29264,7 @@ func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) U return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &Slot{} + m.Slot = SlotFromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25568,7 +29300,7 @@ func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) U return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &Propagation{} + m.Propagation = PropagationFromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25655,7 +29387,7 @@ func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25691,7 +29423,7 @@ func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25727,7 +29459,7 @@ func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25814,7 +29546,7 @@ func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Contribution == nil { - m.Contribution = &ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData{} + m.Contribution = ClientMeta_AdditionalEthV1EventsContributionAndProofContributionDataFromVTPool() } if err := m.Contribution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25901,7 +29633,7 @@ func (m *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) UnmarshalVT return io.ErrUnexpectedEOF } if m.Contribution == nil { - m.Contribution = &ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data{} + m.Contribution = ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2DataFromVTPool() } if err := m.Contribution.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -25988,7 +29720,7 @@ func (m *ClientMeta_ForkChoiceSnapshot) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.RequestEpoch == nil { - m.RequestEpoch = &Epoch{} + m.RequestEpoch = EpochFromVTPool() } if err := m.RequestEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26024,7 +29756,7 @@ func (m *ClientMeta_ForkChoiceSnapshot) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.RequestSlot == nil { - m.RequestSlot = &Slot{} + m.RequestSlot = SlotFromVTPool() } if err := m.RequestSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26185,7 +29917,7 @@ func (m *ClientMeta_ForkChoiceSnapshotV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.RequestEpoch == nil { - m.RequestEpoch = &EpochV2{} + m.RequestEpoch = EpochV2FromVTPool() } if err := m.RequestEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26221,7 +29953,7 @@ func (m *ClientMeta_ForkChoiceSnapshotV2) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.RequestSlot == nil { - m.RequestSlot = &SlotV2{} + m.RequestSlot = SlotV2FromVTPool() } if err := m.RequestSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26416,7 +30148,7 @@ func (m *ClientMeta_AdditionalEthV1DebugForkChoiceData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Snapshot == nil { - m.Snapshot = &ClientMeta_ForkChoiceSnapshot{} + m.Snapshot = ClientMeta_ForkChoiceSnapshotFromVTPool() } if err := m.Snapshot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26503,7 +30235,7 @@ func (m *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Snapshot == nil { - m.Snapshot = &ClientMeta_ForkChoiceSnapshotV2{} + m.Snapshot = ClientMeta_ForkChoiceSnapshotV2FromVTPool() } if err := m.Snapshot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26590,7 +30322,7 @@ func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) UnmarshalVT(dAtA [] return io.ErrUnexpectedEOF } if m.Before == nil { - m.Before = &ClientMeta_ForkChoiceSnapshot{} + m.Before = ClientMeta_ForkChoiceSnapshotFromVTPool() } if err := m.Before.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26626,7 +30358,7 @@ func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) UnmarshalVT(dAtA [] return io.ErrUnexpectedEOF } if m.After == nil { - m.After = &ClientMeta_ForkChoiceSnapshot{} + m.After = ClientMeta_ForkChoiceSnapshotFromVTPool() } if err := m.After.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26713,7 +30445,7 @@ func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) UnmarshalVT(dAtA return io.ErrUnexpectedEOF } if m.Before == nil { - m.Before = &ClientMeta_ForkChoiceSnapshotV2{} + m.Before = ClientMeta_ForkChoiceSnapshotV2FromVTPool() } if err := m.Before.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26749,7 +30481,7 @@ func (m *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) UnmarshalVT(dAtA return io.ErrUnexpectedEOF } if m.After == nil { - m.After = &ClientMeta_ForkChoiceSnapshotV2{} + m.After = ClientMeta_ForkChoiceSnapshotV2FromVTPool() } if err := m.After.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26836,7 +30568,7 @@ func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26872,7 +30604,7 @@ func (m *ClientMeta_AdditionalEthV1BeaconCommitteeData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26991,7 +30723,7 @@ func (m *ClientMeta_AdditionalEthV1BeaconSyncCommitteeData) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -27114,7 +30846,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -28161,7 +31893,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockData) UnmarshalVT(dAtA []byte) err return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &Epoch{} + m.Epoch = EpochFromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -28197,7 +31929,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockData) UnmarshalVT(dAtA []byte) err return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &Slot{} + m.Slot = SlotFromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -28386,7 +32118,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -28422,7 +32154,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockV2Data) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -28773,7 +32505,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) UnmarshalVT( return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -28860,7 +32592,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) UnmarshalVT( return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -28947,7 +32679,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29034,7 +32766,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockDepositData) UnmarshalVT(dAtA []by return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29121,7 +32853,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) Unmarsha return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29208,7 +32940,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) Unmarsha return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29459,7 +33191,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) UnmarshalVT(dAtA [ return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29705,7 +33437,7 @@ func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Source == nil { - m.Source = &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + m.Source = ClientMeta_AdditionalEthV1AttestationSourceV2DataFromVTPool() } if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29741,7 +33473,7 @@ func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Target == nil { - m.Target = &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + m.Target = ClientMeta_AdditionalEthV1AttestationTargetV2DataFromVTPool() } if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29777,7 +33509,7 @@ func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29813,7 +33545,7 @@ func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29849,7 +33581,7 @@ func (m *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Snapshot == nil { - m.Snapshot = &ClientMeta_AttestationDataSnapshot{} + m.Snapshot = ClientMeta_AttestationDataSnapshotFromVTPool() } if err := m.Snapshot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29936,7 +33668,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -29972,7 +33704,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30008,7 +33740,7 @@ func (m *ClientMeta_AdditionalEthV1EventsBlobSidecarData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30095,7 +33827,7 @@ func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) UnmarshalVT(dAtA return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30131,7 +33863,7 @@ func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) UnmarshalVT(dAtA return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30167,7 +33899,7 @@ func (m *ClientMeta_AdditionalEthV1EventsDataColumnSidecarData) UnmarshalVT(dAtA return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30254,7 +33986,7 @@ func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30290,7 +34022,7 @@ func (m *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) UnmarshalVT(dAtA []byt return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30481,7 +34213,7 @@ func (m *ClientMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Source == nil { - m.Source = &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + m.Source = ClientMeta_AdditionalEthV1AttestationSourceV2DataFromVTPool() } if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30517,7 +34249,7 @@ func (m *ClientMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Target == nil { - m.Target = &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + m.Target = ClientMeta_AdditionalEthV1AttestationTargetV2DataFromVTPool() } if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30553,7 +34285,7 @@ func (m *ClientMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30589,7 +34321,7 @@ func (m *ClientMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30625,7 +34357,7 @@ func (m *ClientMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30661,7 +34393,7 @@ func (m *ClientMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.AttestingValidator == nil { - m.AttestingValidator = &AttestingValidatorV2{} + m.AttestingValidator = AttestingValidatorV2FromVTPool() } if err := m.AttestingValidator.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30697,7 +34429,7 @@ func (m *ClientMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Peer == nil { - m.Peer = &libp2p.Peer{} + m.Peer = libp2p.PeerFromVTPool() } if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30856,7 +34588,7 @@ func (m *ClientMeta_AdditionalEthV1ProposerDutyData) UnmarshalVT(dAtA []byte) er return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -30892,7 +34624,7 @@ func (m *ClientMeta_AdditionalEthV1ProposerDutyData) UnmarshalVT(dAtA []byte) er return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31011,7 +34743,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Block == nil { - m.Block = &BlockIdentifier{} + m.Block = BlockIdentifierFromVTPool() } if err := m.Block.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31083,7 +34815,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31119,7 +34851,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31155,7 +34887,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Source == nil { - m.Source = &ClientMeta_AdditionalEthV1AttestationSourceV2Data{} + m.Source = ClientMeta_AdditionalEthV1AttestationSourceV2DataFromVTPool() } if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31191,7 +34923,7 @@ func (m *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Target == nil { - m.Target = &ClientMeta_AdditionalEthV1AttestationTargetV2Data{} + m.Target = ClientMeta_AdditionalEthV1AttestationTargetV2DataFromVTPool() } if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31278,7 +35010,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceAddPeerData) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31365,7 +35097,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRemovePeerData) UnmarshalVT(dAtA []byte return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31452,7 +35184,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRecvRPCData) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31539,7 +35271,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceSendRPCData) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31626,7 +35358,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceDropRPCData) UnmarshalVT(dAtA []byte) e return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31713,7 +35445,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData) UnmarshalVT(dA return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31800,7 +35532,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData) UnmarshalVT(dA return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31887,7 +35619,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData) UnmarshalV return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -31974,7 +35706,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData) UnmarshalVT(dA return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32061,7 +35793,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData) UnmarshalVT(dA return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32148,7 +35880,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceJoinData) UnmarshalVT(dAtA []byte) erro return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32235,7 +35967,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceLeaveData) UnmarshalVT(dAtA []byte) err return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32322,7 +36054,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGraftData) UnmarshalVT(dAtA []byte) err return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32409,7 +36141,7 @@ func (m *ClientMeta_AdditionalLibP2PTracePruneData) UnmarshalVT(dAtA []byte) err return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32496,7 +36228,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceDuplicateMessageData) UnmarshalVT(dAtA return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32583,7 +36315,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceDeliverMessageData) UnmarshalVT(dAtA [] return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32670,7 +36402,7 @@ func (m *ClientMeta_AdditionalLibP2PTracePublishMessageData) UnmarshalVT(dAtA [] return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32757,7 +36489,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRejectMessageData) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32844,7 +36576,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceConnectedData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -32931,7 +36663,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceDisconnectedData) UnmarshalVT(dAtA []by return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33018,7 +36750,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33105,7 +36837,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) UnmarshalVT(dAtA [] return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33192,7 +36924,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceHandleStatusData) UnmarshalVT(dAtA []by return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33279,7 +37011,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceIdentifyData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33366,7 +37098,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) Unmarsha return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33402,7 +37134,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) Unmarsha return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33438,7 +37170,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) Unmarsha return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33474,7 +37206,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) Unmarsha return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33510,7 +37242,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData) Unmarsha return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33597,7 +37329,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData) UnmarshalVT(dA return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33684,7 +37416,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData) UnmarshalVT(dAtA [] return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33771,7 +37503,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33807,7 +37539,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33843,7 +37575,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33879,7 +37611,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33915,7 +37647,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -33951,7 +37683,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34146,7 +37878,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) U return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34233,7 +37965,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) U return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34320,7 +38052,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Source == nil { - m.Source = &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData{} + m.Source = ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceDataFromVTPool() } if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34356,7 +38088,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Target == nil { - m.Target = &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData{} + m.Target = ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetDataFromVTPool() } if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34392,7 +38124,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34428,7 +38160,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34464,7 +38196,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34500,7 +38232,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.AttestingValidator == nil { - m.AttestingValidator = &AttestingValidatorV2{} + m.AttestingValidator = AttestingValidatorV2FromVTPool() } if err := m.AttestingValidator.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34536,7 +38268,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34572,7 +38304,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34608,7 +38340,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Unmarsh return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34803,7 +38535,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) Unmarsh return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34839,7 +38571,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) Unmarsh return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34875,7 +38607,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) Unmarsh return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34911,7 +38643,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) Unmarsh return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -34947,7 +38679,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) Unmarsh return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35019,7 +38751,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData) Unmarsh return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35214,7 +38946,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35250,7 +38982,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35286,7 +39018,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35322,7 +39054,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35358,7 +39090,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35394,7 +39126,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) UnmarshalVT(d return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35589,7 +39321,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) Unmarsh return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35625,7 +39357,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) Unmarsh return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35661,7 +39393,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) Unmarsh return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35697,7 +39429,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) Unmarsh return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35733,7 +39465,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) Unmarsh return io.ErrUnexpectedEOF } if m.Propagation == nil { - m.Propagation = &PropagationV2{} + m.Propagation = PropagationV2FromVTPool() } if err := m.Propagation.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35769,7 +39501,7 @@ func (m *ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData) Unmarsh return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = &libp2p.TraceEventMetadata{} + m.Metadata = libp2p.TraceEventMetadataFromVTPool() } if err := m.Metadata.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -35964,7 +39696,7 @@ func (m *ClientMeta_AdditionalEthV1ValidatorsData) UnmarshalVT(dAtA []byte) erro return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36051,7 +39783,7 @@ func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) Unmars return io.ErrUnexpectedEOF } if m.Relay == nil { - m.Relay = &mevrelay.Relay{} + m.Relay = mevrelay.RelayFromVTPool() } if err := m.Relay.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36087,7 +39819,7 @@ func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) Unmars return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36123,7 +39855,7 @@ func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) Unmars return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36159,7 +39891,7 @@ func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) Unmars return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36195,7 +39927,7 @@ func (m *ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData) Unmars return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36354,7 +40086,7 @@ func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Relay == nil { - m.Relay = &mevrelay.Relay{} + m.Relay = mevrelay.RelayFromVTPool() } if err := m.Relay.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36390,7 +40122,7 @@ func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36426,7 +40158,7 @@ func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36462,7 +40194,7 @@ func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36498,7 +40230,7 @@ func (m *ClientMeta_AdditionalMevRelayPayloadDeliveredData) UnmarshalVT(dAtA []b return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36657,7 +40389,7 @@ func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36693,7 +40425,7 @@ func (m *ClientMeta_AdditionalEthV3ValidatorBlockData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37128,7 +40860,7 @@ func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Relay == nil { - m.Relay = &mevrelay.Relay{} + m.Relay = mevrelay.RelayFromVTPool() } if err := m.Relay.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37164,7 +40896,7 @@ func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37200,7 +40932,7 @@ func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.WallclockSlot == nil { - m.WallclockSlot = &SlotV2{} + m.WallclockSlot = SlotV2FromVTPool() } if err := m.WallclockSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37236,7 +40968,7 @@ func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37272,7 +41004,7 @@ func (m *ClientMeta_AdditionalMevRelayValidatorRegistrationData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.WallclockEpoch == nil { - m.WallclockEpoch = &EpochV2{} + m.WallclockEpoch = EpochV2FromVTPool() } if err := m.WallclockEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37395,7 +41127,7 @@ func (m *ClientMeta_AdditionalNodeRecordConsensusData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.FinalizedEpoch == nil { - m.FinalizedEpoch = &EpochV2{} + m.FinalizedEpoch = EpochV2FromVTPool() } if err := m.FinalizedEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37431,7 +41163,7 @@ func (m *ClientMeta_AdditionalNodeRecordConsensusData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.HeadSlot == nil { - m.HeadSlot = &SlotV2{} + m.HeadSlot = SlotV2FromVTPool() } if err := m.HeadSlot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37467,7 +41199,7 @@ func (m *ClientMeta_AdditionalNodeRecordConsensusData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.HeadEpoch == nil { - m.HeadEpoch = &EpochV2{} + m.HeadEpoch = EpochV2FromVTPool() } if err := m.HeadEpoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37554,7 +41286,7 @@ func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) UnmarshalVT(dAtA return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37590,7 +41322,7 @@ func (m *ClientMeta_AdditionalConsensusEngineAPINewPayloadData) UnmarshalVT(dAtA return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37677,7 +41409,7 @@ func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) UnmarshalVT(dAtA [ return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37713,7 +41445,7 @@ func (m *ClientMeta_AdditionalConsensusEngineAPIGetBlobsData) UnmarshalVT(dAtA [ return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37800,7 +41532,7 @@ func (m *ClientMeta_AdditionalEthV1BeaconBlobData) UnmarshalVT(dAtA []byte) erro return io.ErrUnexpectedEOF } if m.Epoch == nil { - m.Epoch = &EpochV2{} + m.Epoch = EpochV2FromVTPool() } if err := m.Epoch.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -37836,7 +41568,7 @@ func (m *ClientMeta_AdditionalEthV1BeaconBlobData) UnmarshalVT(dAtA []byte) erro return io.ErrUnexpectedEOF } if m.Slot == nil { - m.Slot = &SlotV2{} + m.Slot = SlotV2FromVTPool() } if err := m.Slot.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -38102,7 +41834,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Ethereum == nil { - m.Ethereum = &ClientMeta_Ethereum{} + m.Ethereum = ClientMeta_EthereumFromVTPool() } if err := m.Ethereum.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -38269,7 +42001,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsAttestationData{} + v := ClientMeta_AdditionalEthV1EventsAttestationDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38310,7 +42042,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsHeadData{} + v := ClientMeta_AdditionalEthV1EventsHeadDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38351,7 +42083,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsBlockData{} + v := ClientMeta_AdditionalEthV1EventsBlockDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38392,7 +42124,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsVoluntaryExitData{} + v := ClientMeta_AdditionalEthV1EventsVoluntaryExitDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38433,7 +42165,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData{} + v := ClientMeta_AdditionalEthV1EventsFinalizedCheckpointDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38474,7 +42206,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsChainReorgData{} + v := ClientMeta_AdditionalEthV1EventsChainReorgDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38515,7 +42247,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsContributionAndProofData{} + v := ClientMeta_AdditionalEthV1EventsContributionAndProofDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38556,7 +42288,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalMempoolTransactionData{} + v := ClientMeta_AdditionalMempoolTransactionDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38597,7 +42329,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockData{} + v := ClientMeta_AdditionalEthV2BeaconBlockDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38638,7 +42370,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1DebugForkChoiceData{} + v := ClientMeta_AdditionalEthV1DebugForkChoiceDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38679,7 +42411,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData{} + v := ClientMeta_AdditionalEthV1DebugForkChoiceReOrgDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38720,7 +42452,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1BeaconCommitteeData{} + v := ClientMeta_AdditionalEthV1BeaconCommitteeDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38761,7 +42493,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1ValidatorAttestationDataData{} + v := ClientMeta_AdditionalEthV1ValidatorAttestationDataDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38802,7 +42534,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsAttestationV2Data{} + v := ClientMeta_AdditionalEthV1EventsAttestationV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38843,7 +42575,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsHeadV2Data{} + v := ClientMeta_AdditionalEthV1EventsHeadV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38884,7 +42616,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsBlockV2Data{} + v := ClientMeta_AdditionalEthV1EventsBlockV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38925,7 +42657,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data{} + v := ClientMeta_AdditionalEthV1EventsVoluntaryExitV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -38966,7 +42698,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data{} + v := ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39007,7 +42739,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsChainReorgV2Data{} + v := ClientMeta_AdditionalEthV1EventsChainReorgV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39048,7 +42780,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data{} + v := ClientMeta_AdditionalEthV1EventsContributionAndProofV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39089,7 +42821,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalMempoolTransactionV2Data{} + v := ClientMeta_AdditionalMempoolTransactionV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39130,7 +42862,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockV2Data{} + v := ClientMeta_AdditionalEthV2BeaconBlockV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39171,7 +42903,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1DebugForkChoiceV2Data{} + v := ClientMeta_AdditionalEthV1DebugForkChoiceV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39212,7 +42944,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data{} + v := ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2DataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39253,7 +42985,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData{} + v := ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39294,7 +43026,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData{} + v := ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39335,7 +43067,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData{} + v := ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39376,7 +43108,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockDepositData{} + v := ClientMeta_AdditionalEthV2BeaconBlockDepositDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39417,7 +43149,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData{} + v := ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39458,7 +43190,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData{} + v := ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39499,7 +43231,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData{} + v := ClientMeta_AdditionalEthV2BeaconBlockWithdrawalDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39540,7 +43272,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsBlobSidecarData{} + v := ClientMeta_AdditionalEthV1EventsBlobSidecarDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39581,7 +43313,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1BeaconBlobSidecarData{} + v := ClientMeta_AdditionalEthV1BeaconBlobSidecarDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39622,7 +43354,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalBeaconP2PAttestationData{} + v := ClientMeta_AdditionalBeaconP2PAttestationDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39663,7 +43395,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1ProposerDutyData{} + v := ClientMeta_AdditionalEthV1ProposerDutyDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39704,7 +43436,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData{} + v := ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39745,7 +43477,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceAddPeerData{} + v := ClientMeta_AdditionalLibP2PTraceAddPeerDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39786,7 +43518,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRemovePeerData{} + v := ClientMeta_AdditionalLibP2PTraceRemovePeerDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39827,7 +43559,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRecvRPCData{} + v := ClientMeta_AdditionalLibP2PTraceRecvRPCDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39868,7 +43600,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceSendRPCData{} + v := ClientMeta_AdditionalLibP2PTraceSendRPCDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39909,7 +43641,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceJoinData{} + v := ClientMeta_AdditionalLibP2PTraceJoinDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39950,7 +43682,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceConnectedData{} + v := ClientMeta_AdditionalLibP2PTraceConnectedDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39991,7 +43723,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceDisconnectedData{} + v := ClientMeta_AdditionalLibP2PTraceDisconnectedDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40032,7 +43764,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceHandleMetadataData{} + v := ClientMeta_AdditionalLibP2PTraceHandleMetadataDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40073,7 +43805,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceHandleStatusData{} + v := ClientMeta_AdditionalLibP2PTraceHandleStatusDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40114,7 +43846,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData{} + v := ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40155,7 +43887,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData{} + v := ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40196,7 +43928,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData{} + v := ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40237,7 +43969,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1ValidatorsData{} + v := ClientMeta_AdditionalEthV1ValidatorsDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40278,7 +44010,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionData{} + v := ClientMeta_AdditionalMevRelayBidTraceBuilderBlockSubmissionDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40319,7 +44051,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalMevRelayPayloadDeliveredData{} + v := ClientMeta_AdditionalMevRelayPayloadDeliveredDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40411,7 +44143,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV3ValidatorBlockData{} + v := ClientMeta_AdditionalEthV3ValidatorBlockDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40452,7 +44184,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalMevRelayValidatorRegistrationData{} + v := ClientMeta_AdditionalMevRelayValidatorRegistrationDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40493,7 +44225,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsBlockGossipData{} + v := ClientMeta_AdditionalEthV1EventsBlockGossipDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40534,7 +44266,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceDropRPCData{} + v := ClientMeta_AdditionalLibP2PTraceDropRPCDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40575,7 +44307,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceLeaveData{} + v := ClientMeta_AdditionalLibP2PTraceLeaveDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40616,7 +44348,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceGraftData{} + v := ClientMeta_AdditionalLibP2PTraceGraftDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40657,7 +44389,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTracePruneData{} + v := ClientMeta_AdditionalLibP2PTracePruneDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40698,7 +44430,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceDuplicateMessageData{} + v := ClientMeta_AdditionalLibP2PTraceDuplicateMessageDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40739,7 +44471,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceDeliverMessageData{} + v := ClientMeta_AdditionalLibP2PTraceDeliverMessageDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40780,7 +44512,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTracePublishMessageData{} + v := ClientMeta_AdditionalLibP2PTracePublishMessageDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40821,7 +44553,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRejectMessageData{} + v := ClientMeta_AdditionalLibP2PTraceRejectMessageDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40862,7 +44594,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveData{} + v := ClientMeta_AdditionalLibP2PTraceRPCMetaControlIHaveDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40903,7 +44635,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantData{} + v := ClientMeta_AdditionalLibP2PTraceRPCMetaControlIWantDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40944,7 +44676,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantData{} + v := ClientMeta_AdditionalLibP2PTraceRPCMetaControlIDontWantDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -40985,7 +44717,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftData{} + v := ClientMeta_AdditionalLibP2PTraceRPCMetaControlGraftDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41026,7 +44758,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneData{} + v := ClientMeta_AdditionalLibP2PTraceRPCMetaControlPruneDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41067,7 +44799,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionData{} + v := ClientMeta_AdditionalLibP2PTraceRPCMetaSubscriptionDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41108,7 +44840,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRPCMetaMessageData{} + v := ClientMeta_AdditionalLibP2PTraceRPCMetaMessageDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41149,7 +44881,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalNodeRecordConsensusData{} + v := ClientMeta_AdditionalNodeRecordConsensusDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41190,7 +44922,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofData{} + v := ClientMeta_AdditionalLibP2PTraceGossipSubAggregateAndProofDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41231,7 +44963,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1EventsDataColumnSidecarData{} + v := ClientMeta_AdditionalEthV1EventsDataColumnSidecarDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41272,7 +45004,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarData{} + v := ClientMeta_AdditionalLibP2PTraceGossipSubDataColumnSidecarDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41313,7 +45045,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatData{} + v := ClientMeta_AdditionalLibP2PTraceSyntheticHeartbeatDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41354,7 +45086,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData{} + v := ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41395,7 +45127,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalConsensusEngineAPINewPayloadData{} + v := ClientMeta_AdditionalConsensusEngineAPINewPayloadDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41436,7 +45168,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalConsensusEngineAPIGetBlobsData{} + v := ClientMeta_AdditionalConsensusEngineAPIGetBlobsDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41477,7 +45209,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1BeaconBlobData{} + v := ClientMeta_AdditionalEthV1BeaconBlobDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41518,7 +45250,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV1BeaconSyncCommitteeData{} + v := ClientMeta_AdditionalEthV1BeaconSyncCommitteeDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41559,7 +45291,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateData{} + v := ClientMeta_AdditionalEthV2BeaconBlockSyncAggregateDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -41600,7 +45332,7 @@ func (m *ClientMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ClientMeta_AdditionalLibP2PTraceIdentifyData{} + v := ClientMeta_AdditionalLibP2PTraceIdentifyDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -42059,7 +45791,7 @@ func (m *ServerMeta_Client) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Geo == nil { - m.Geo = &ServerMeta_Geo{} + m.Geo = ServerMeta_GeoFromVTPool() } if err := m.Geo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42210,7 +45942,7 @@ func (m *ServerMeta_Peer) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Geo == nil { - m.Geo = &ServerMeta_Geo{} + m.Geo = ServerMeta_GeoFromVTPool() } if err := m.Geo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42297,7 +46029,7 @@ func (m *ServerMeta_AdditionalBeaconP2PAttestationData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Peer == nil { - m.Peer = &ServerMeta_Peer{} + m.Peer = ServerMeta_PeerFromVTPool() } if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42384,7 +46116,7 @@ func (m *ServerMeta_AdditionalLibp2PTraceConnectedData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Peer == nil { - m.Peer = &ServerMeta_Peer{} + m.Peer = ServerMeta_PeerFromVTPool() } if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42471,7 +46203,7 @@ func (m *ServerMeta_AdditionalLibp2PTraceDisconnectedData) UnmarshalVT(dAtA []by return io.ErrUnexpectedEOF } if m.Peer == nil { - m.Peer = &ServerMeta_Peer{} + m.Peer = ServerMeta_PeerFromVTPool() } if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42558,7 +46290,7 @@ func (m *ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData) UnmarshalVT(dAt return io.ErrUnexpectedEOF } if m.Peer == nil { - m.Peer = &ServerMeta_Peer{} + m.Peer = ServerMeta_PeerFromVTPool() } if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42645,7 +46377,7 @@ func (m *ServerMeta_AdditionalLibp2PTraceIdentifyData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Peer == nil { - m.Peer = &ServerMeta_Peer{} + m.Peer = ServerMeta_PeerFromVTPool() } if err := m.Peer.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42732,7 +46464,7 @@ func (m *ServerMeta_AdditionalNodeRecordConsensusData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Geo == nil { - m.Geo = &ServerMeta_Geo{} + m.Geo = ServerMeta_GeoFromVTPool() } if err := m.Geo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42819,7 +46551,7 @@ func (m *ServerMeta_AdditionalNodeRecordExecutionData) UnmarshalVT(dAtA []byte) return io.ErrUnexpectedEOF } if m.Geo == nil { - m.Geo = &ServerMeta_Geo{} + m.Geo = ServerMeta_GeoFromVTPool() } if err := m.Geo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42906,7 +46638,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Event == nil { - m.Event = &ServerMeta_Event{} + m.Event = ServerMeta_EventFromVTPool() } if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42942,7 +46674,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Client == nil { - m.Client = &ServerMeta_Client{} + m.Client = ServerMeta_ClientFromVTPool() } if err := m.Client.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -42982,7 +46714,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ServerMeta_AdditionalBeaconP2PAttestationData{} + v := ServerMeta_AdditionalBeaconP2PAttestationDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -43023,7 +46755,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ServerMeta_AdditionalLibp2PTraceConnectedData{} + v := ServerMeta_AdditionalLibp2PTraceConnectedDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -43064,7 +46796,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ServerMeta_AdditionalLibp2PTraceDisconnectedData{} + v := ServerMeta_AdditionalLibp2PTraceDisconnectedDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -43105,7 +46837,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ServerMeta_AdditionalNodeRecordConsensusData{} + v := ServerMeta_AdditionalNodeRecordConsensusDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -43146,7 +46878,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ServerMeta_AdditionalNodeRecordExecutionData{} + v := ServerMeta_AdditionalNodeRecordExecutionDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -43187,7 +46919,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatData{} + v := ServerMeta_AdditionalLibP2PTraceSyntheticHeartbeatDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -43228,7 +46960,7 @@ func (m *ServerMeta) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ServerMeta_AdditionalLibp2PTraceIdentifyData{} + v := ServerMeta_AdditionalLibp2PTraceIdentifyDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -43316,7 +47048,7 @@ func (m *Meta) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Client == nil { - m.Client = &ClientMeta{} + m.Client = ClientMetaFromVTPool() } if err := m.Client.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -43352,7 +47084,7 @@ func (m *Meta) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Server == nil { - m.Server = &ServerMeta{} + m.Server = ServerMetaFromVTPool() } if err := m.Server.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -44817,7 +48549,7 @@ func (m *ExecutionBlockMetrics) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.StateReads == nil { - m.StateReads = &ExecutionBlockMetrics_StateReads{} + m.StateReads = ExecutionBlockMetrics_StateReadsFromVTPool() } if err := m.StateReads.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -44853,7 +48585,7 @@ func (m *ExecutionBlockMetrics) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.StateWrites == nil { - m.StateWrites = &ExecutionBlockMetrics_StateWrites{} + m.StateWrites = ExecutionBlockMetrics_StateWritesFromVTPool() } if err := m.StateWrites.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -44889,7 +48621,7 @@ func (m *ExecutionBlockMetrics) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.AccountCache == nil { - m.AccountCache = &ExecutionBlockMetrics_CacheEntry{} + m.AccountCache = ExecutionBlockMetrics_CacheEntryFromVTPool() } if err := m.AccountCache.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -44925,7 +48657,7 @@ func (m *ExecutionBlockMetrics) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.StorageCache == nil { - m.StorageCache = &ExecutionBlockMetrics_CacheEntry{} + m.StorageCache = ExecutionBlockMetrics_CacheEntryFromVTPool() } if err := m.StorageCache.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -44961,7 +48693,7 @@ func (m *ExecutionBlockMetrics) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.CodeCache == nil { - m.CodeCache = &ExecutionBlockMetrics_CodeCacheEntry{} + m.CodeCache = ExecutionBlockMetrics_CodeCacheEntryFromVTPool() } if err := m.CodeCache.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -45048,7 +48780,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Event == nil { - m.Event = &Event{} + m.Event = EventFromVTPool() } if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -45084,7 +48816,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Meta == nil { - m.Meta = &Meta{} + m.Meta = MetaFromVTPool() } if err := m.Meta.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -45124,7 +48856,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.Attestation{} + v := v1.AttestationFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45165,7 +48897,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventBlock{} + v := v1.EventBlockFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45206,7 +48938,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventChainReorg{} + v := v1.EventChainReorgFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45247,7 +48979,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventFinalizedCheckpoint{} + v := v1.EventFinalizedCheckpointFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45288,7 +49020,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventHead{} + v := v1.EventHeadFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45329,7 +49061,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventVoluntaryExit{} + v := v1.EventVoluntaryExitFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45370,7 +49102,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventContributionAndProof{} + v := v1.EventContributionAndProofFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45443,7 +49175,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v2.EventBlock{} + v := v2.EventBlockFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45484,7 +49216,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.ForkChoice{} + v := v1.ForkChoiceFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45525,7 +49257,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &DebugForkChoiceReorg{} + v := DebugForkChoiceReorgFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45566,7 +49298,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.Committee{} + v := v1.CommitteeFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45607,7 +49339,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.AttestationDataV2{} + v := v1.AttestationDataV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45648,7 +49380,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.AttestationV2{} + v := v1.AttestationV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45689,7 +49421,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventBlockV2{} + v := v1.EventBlockV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45730,7 +49462,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventChainReorgV2{} + v := v1.EventChainReorgV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45771,7 +49503,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventFinalizedCheckpointV2{} + v := v1.EventFinalizedCheckpointV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45812,7 +49544,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventHeadV2{} + v := v1.EventHeadV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45853,7 +49585,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventVoluntaryExitV2{} + v := v1.EventVoluntaryExitV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45894,7 +49626,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventContributionAndProofV2{} + v := v1.EventContributionAndProofV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -45967,7 +49699,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v2.EventBlockV2{} + v := v2.EventBlockV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46008,7 +49740,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.ForkChoiceV2{} + v := v1.ForkChoiceV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46049,7 +49781,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &DebugForkChoiceReorgV2{} + v := DebugForkChoiceReorgV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46090,7 +49822,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.AttesterSlashingV2{} + v := v1.AttesterSlashingV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46131,7 +49863,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.ProposerSlashingV2{} + v := v1.ProposerSlashingV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46172,7 +49904,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.SignedVoluntaryExitV2{} + v := v1.SignedVoluntaryExitV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46213,7 +49945,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.DepositV2{} + v := v1.DepositV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46254,7 +49986,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v2.SignedBLSToExecutionChangeV2{} + v := v2.SignedBLSToExecutionChangeV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46295,7 +50027,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.Transaction{} + v := v1.TransactionFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46336,7 +50068,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.WithdrawalV2{} + v := v1.WithdrawalV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46377,7 +50109,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventBlobSidecar{} + v := v1.EventBlobSidecarFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46418,7 +50150,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.BlobSidecar{} + v := v1.BlobSidecarFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46459,7 +50191,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.AttestationV2{} + v := v1.AttestationV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46500,7 +50232,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.ProposerDuty{} + v := v1.ProposerDutyFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46541,7 +50273,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.ElaboratedAttestation{} + v := v1.ElaboratedAttestationFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46582,7 +50314,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.AddPeer{} + v := libp2p.AddPeerFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46623,7 +50355,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.RemovePeer{} + v := libp2p.RemovePeerFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46664,7 +50396,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.RecvRPC{} + v := libp2p.RecvRPCFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46705,7 +50437,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.SendRPC{} + v := libp2p.SendRPCFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46746,7 +50478,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.Join{} + v := libp2p.JoinFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46787,7 +50519,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.Connected{} + v := libp2p.ConnectedFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46828,7 +50560,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.Disconnected{} + v := libp2p.DisconnectedFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46869,7 +50601,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.HandleMetadata{} + v := libp2p.HandleMetadataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46910,7 +50642,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.HandleStatus{} + v := libp2p.HandleStatusFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46951,7 +50683,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &gossipsub.BeaconBlock{} + v := gossipsub.BeaconBlockFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -46992,7 +50724,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.Attestation{} + v := v1.AttestationFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47033,7 +50765,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &gossipsub.BlobSidecar{} + v := gossipsub.BlobSidecarFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47074,7 +50806,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &Validators{} + v := ValidatorsFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47115,7 +50847,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &mevrelay.BidTrace{} + v := mevrelay.BidTraceFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47156,7 +50888,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &mevrelay.ProposerPayloadDelivered{} + v := mevrelay.ProposerPayloadDeliveredFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47197,7 +50929,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v2.EventBlockV2{} + v := v2.EventBlockV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47238,7 +50970,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &mevrelay.ValidatorRegistration{} + v := mevrelay.ValidatorRegistrationFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47279,7 +51011,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventBlockGossip{} + v := v1.EventBlockGossipFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47320,7 +51052,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.DropRPC{} + v := libp2p.DropRPCFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47361,7 +51093,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.Leave{} + v := libp2p.LeaveFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47402,7 +51134,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.Graft{} + v := libp2p.GraftFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47443,7 +51175,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.Prune{} + v := libp2p.PruneFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47484,7 +51216,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.DuplicateMessage{} + v := libp2p.DuplicateMessageFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47525,7 +51257,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.DeliverMessage{} + v := libp2p.DeliverMessageFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47566,7 +51298,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.PublishMessage{} + v := libp2p.PublishMessageFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47607,7 +51339,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.RejectMessage{} + v := libp2p.RejectMessageFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47648,7 +51380,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.ControlIHaveMetaItem{} + v := libp2p.ControlIHaveMetaItemFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47689,7 +51421,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.ControlIWantMetaItem{} + v := libp2p.ControlIWantMetaItemFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47730,7 +51462,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.ControlIDontWantMetaItem{} + v := libp2p.ControlIDontWantMetaItemFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47771,7 +51503,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.ControlGraftMetaItem{} + v := libp2p.ControlGraftMetaItemFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47812,7 +51544,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.ControlPruneMetaItem{} + v := libp2p.ControlPruneMetaItemFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47853,7 +51585,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.SubMetaItem{} + v := libp2p.SubMetaItemFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47894,7 +51626,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.MessageMetaItem{} + v := libp2p.MessageMetaItemFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47935,7 +51667,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &noderecord.Consensus{} + v := noderecord.ConsensusFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -47976,7 +51708,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &noderecord.Execution{} + v := noderecord.ExecutionFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48017,7 +51749,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.SignedAggregateAttestationAndProofV2{} + v := v1.SignedAggregateAttestationAndProofV2FromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48058,7 +51790,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.EventDataColumnSidecar{} + v := v1.EventDataColumnSidecarFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48099,7 +51831,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &gossipsub.DataColumnSidecar{} + v := gossipsub.DataColumnSidecarFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48140,7 +51872,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.SyntheticHeartbeat{} + v := libp2p.SyntheticHeartbeatFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48181,7 +51913,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.Identify{} + v := libp2p.IdentifyFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48222,7 +51954,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &libp2p.DataColumnCustodyProbe{} + v := libp2p.DataColumnCustodyProbeFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48263,7 +51995,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ExecutionStateSize{} + v := ExecutionStateSizeFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48304,7 +52036,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ConsensusEngineAPINewPayload{} + v := ConsensusEngineAPINewPayloadFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48345,7 +52077,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ConsensusEngineAPIGetBlobs{} + v := ConsensusEngineAPIGetBlobsFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48386,7 +52118,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ExecutionEngineNewPayload{} + v := ExecutionEngineNewPayloadFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48427,7 +52159,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ExecutionEngineGetBlobs{} + v := ExecutionEngineGetBlobsFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48468,7 +52200,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &v1.Blob{} + v := v1.BlobFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48509,7 +52241,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &SyncCommitteeData{} + v := SyncCommitteeDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48550,7 +52282,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &SyncAggregateData{} + v := SyncAggregateDataFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -48591,7 +52323,7 @@ func (m *DecoratedEvent) UnmarshalVT(dAtA []byte) error { return err } } else { - v := &ExecutionBlockMetrics{} + v := ExecutionBlockMetricsFromVTPool() if err := v.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } From f317b9a7929407366b7758690cf24c31fff77914 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Thu, 26 Feb 2026 09:55:36 +1000 Subject: [PATCH 13/27] feat(consumoor): add event_lag_seconds histogram metric Track the lag between the event's client-side timestamp and the wall clock time at routing, exposed as a per-event-name Prometheus histogram. --- pkg/consumoor/router/engine.go | 9 +++++++++ pkg/consumoor/telemetry/metrics.go | 10 ++++++++++ 2 files changed, 19 insertions(+) diff --git a/pkg/consumoor/router/engine.go b/pkg/consumoor/router/engine.go index 3fc721ca..b1af5565 100644 --- a/pkg/consumoor/router/engine.go +++ b/pkg/consumoor/router/engine.go @@ -1,6 +1,8 @@ package router import ( + "time" + "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/consumoor/telemetry" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -112,6 +114,13 @@ func (r *Engine) Route(event *xatu.DecoratedEvent) Outcome { for _, result := range results { r.metrics.MessagesRouted().WithLabelValues(eventName.String(), result.Table).Inc() } + + if ts := event.GetEvent().GetDateTime(); ts != nil { + lag := time.Since(ts.AsTime()).Seconds() + if lag >= 0 { + r.metrics.EventLag().WithLabelValues(eventName.String()).Observe(lag) + } + } } return Outcome{ diff --git a/pkg/consumoor/telemetry/metrics.go b/pkg/consumoor/telemetry/metrics.go index ec364eb7..adef6d99 100644 --- a/pkg/consumoor/telemetry/metrics.go +++ b/pkg/consumoor/telemetry/metrics.go @@ -19,6 +19,7 @@ type Metrics struct { writeDuration *prometheus.HistogramVec batchSize *prometheus.HistogramVec flattenErrors *prometheus.CounterVec + eventLag *prometheus.HistogramVec activeTopics prometheus.Gauge kafkaConsumerLag *prometheus.GaugeVec outputMaxInFlight prometheus.Gauge @@ -136,6 +137,14 @@ func NewMetrics(namespace string) *Metrics { Help: "Total number of flattener errors.", }, []string{"event_name", "table"}), + eventLag: promauto.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "event_lag_seconds", + Help: "Lag between the event timestamp and the wall clock time at routing.", + Buckets: []float64{0.1, 0.5, 1, 2, 5, 10, 30, 60, 120, 300, 600}, + }, []string{"event_name"}), + activeTopics: promauto.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, @@ -262,6 +271,7 @@ func (m *Metrics) WriteErrors() *prometheus.CounterVec { return m.writeErro func (m *Metrics) WriteDuration() *prometheus.HistogramVec { return m.writeDuration } func (m *Metrics) BatchSize() *prometheus.HistogramVec { return m.batchSize } func (m *Metrics) FlattenErrors() *prometheus.CounterVec { return m.flattenErrors } +func (m *Metrics) EventLag() *prometheus.HistogramVec { return m.eventLag } func (m *Metrics) ActiveTopics() prometheus.Gauge { return m.activeTopics } func (m *Metrics) KafkaConsumerLag() *prometheus.GaugeVec { return m.kafkaConsumerLag } func (m *Metrics) OutputMaxInFlight() prometheus.Gauge { return m.outputMaxInFlight } From 9b141993787f017b342eec15aaa9297889c25a7d Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Thu, 26 Feb 2026 10:10:17 +1000 Subject: [PATCH 14/27] feat(consumoor): increase default outputBatchPeriod to 5s Aligns with production Vector configs where most pipelines use 5s batch timeouts. Larger batches reduce ClickHouse INSERT frequency. --- pkg/consumoor/source/config.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/consumoor/source/config.go b/pkg/consumoor/source/config.go index 166895d9..2c850c79 100644 --- a/pkg/consumoor/source/config.go +++ b/pkg/consumoor/source/config.go @@ -119,8 +119,8 @@ type KafkaConfig struct { OutputBatchCount int `yaml:"outputBatchCount" default:"10000"` // OutputBatchPeriod is the maximum time Benthos waits to fill a batch // before flushing a partial batch. Ensures low-volume topics still make - // progress. Default: 1s. Set to 0 to disable period-based flushing. - OutputBatchPeriod time.Duration `yaml:"outputBatchPeriod" default:"1s"` + // progress. Default: 5s. Set to 0 to disable period-based flushing. + OutputBatchPeriod time.Duration `yaml:"outputBatchPeriod" default:"5s"` // MaxInFlight is the maximum number of concurrent WriteBatch calls // Benthos makes for each stream's output. Higher values increase From 43d49480e12bb3e1f94fa32d127b8b7c8372716f Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Thu, 26 Feb 2026 11:32:07 +1000 Subject: [PATCH 15/27] feat(consumoor): add missing observability metrics and dashboard panels Add write_retries_total and adaptive_limiter_rejections_total counters to surface retry pressure and limiter backpressure. Wire up the existing batch_flush_trigger_total metric in the output path. Update the Grafana dashboard to visualize event lag, batch flush triggers, adaptive limiter state, write retries, and active topics. Remove broken buffer_usage panels that referenced non-existent metrics. --- .../grafana/dashboards/xatu/consumoor.json | 526 ++++++++++++++++-- pkg/consumoor/clickhouse/table_writer.go | 7 +- pkg/consumoor/clickhouse/writer.go | 2 + pkg/consumoor/source/benthos.go | 15 +- pkg/consumoor/source/output.go | 27 +- pkg/consumoor/telemetry/metrics.go | 67 ++- 6 files changed, 577 insertions(+), 67 deletions(-) diff --git a/deploy/local/docker-compose/grafana/dashboards/xatu/consumoor.json b/deploy/local/docker-compose/grafana/dashboards/xatu/consumoor.json index 9fd861ad..3c3ed6df 100644 --- a/deploy/local/docker-compose/grafana/dashboards/xatu/consumoor.json +++ b/deploy/local/docker-compose/grafana/dashboards/xatu/consumoor.json @@ -48,7 +48,7 @@ }, "overrides": [] }, - "gridPos": { "h": 4, "w": 6, "x": 0, "y": 1 }, + "gridPos": { "h": 4, "w": 5, "x": 0, "y": 1 }, "id": 1, "options": { "colorMode": "value", @@ -95,7 +95,7 @@ }, "overrides": [] }, - "gridPos": { "h": 4, "w": 6, "x": 6, "y": 1 }, + "gridPos": { "h": 4, "w": 5, "x": 5, "y": 1 }, "id": 2, "options": { "colorMode": "value", @@ -143,7 +143,7 @@ }, "overrides": [] }, - "gridPos": { "h": 4, "w": 6, "x": 12, "y": 1 }, + "gridPos": { "h": 4, "w": 5, "x": 10, "y": 1 }, "id": 3, "options": { "colorMode": "value", @@ -191,7 +191,7 @@ }, "overrides": [] }, - "gridPos": { "h": 4, "w": 6, "x": 18, "y": 1 }, + "gridPos": { "h": 4, "w": 5, "x": 15, "y": 1 }, "id": 4, "options": { "colorMode": "value", @@ -218,6 +218,52 @@ "title": "Total Consumer Lag", "type": "stat" }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { "mode": "thresholds" }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { "color": "green", "value": null } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { "h": 4, "w": 4, "x": 20, "y": 1 }, + "id": 5, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "targets": [ + { + "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "editorMode": "code", + "expr": "xatu_consumoor_active_topics", + "legendFormat": "Active topics", + "range": true, + "refId": "A" + } + ], + "title": "Active Topics", + "type": "stat" + }, { "collapsed": false, "gridPos": { "h": 1, "w": 24, "x": 0, "y": 5 }, @@ -424,9 +470,75 @@ "title": "Decode Errors by Topic", "type": "timeseries" }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { "mode": "palette-classic" }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { "legend": false, "tooltip": false, "viz": false }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { "type": "linear" }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { "group": "A", "mode": "normal" }, + "thresholdsStyle": { "mode": "off" } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { "color": "green", "value": null } + ] + }, + "unit": "ops" + }, + "overrides": [] + }, + "gridPos": { "h": 8, "w": 24, "x": 0, "y": 14 }, + "id": 13, + "options": { + "legend": { + "calcs": ["lastNotNull"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "sortBy": "Last *", + "sortDesc": true + }, + "tooltip": { "mode": "multi", "sort": "desc" } + }, + "targets": [ + { + "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "editorMode": "code", + "expr": "sum by(topic, trigger) (rate(xatu_consumoor_batch_flush_trigger_total[$__rate_interval]))", + "legendFormat": "{{topic}} ({{trigger}})", + "range": true, + "refId": "A" + } + ], + "title": "Batch Flush Triggers", + "type": "timeseries" + }, { "collapsed": false, - "gridPos": { "h": 1, "w": 24, "x": 0, "y": 14 }, + "gridPos": { "h": 1, "w": 24, "x": 0, "y": 22 }, "id": 300, "title": "Routing", "type": "row" @@ -471,7 +583,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 0, "y": 15 }, + "gridPos": { "h": 8, "w": 8, "x": 0, "y": 23 }, "id": 20, "options": { "legend": { @@ -538,7 +650,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 8, "y": 15 }, + "gridPos": { "h": 8, "w": 8, "x": 8, "y": 23 }, "id": 21, "options": { "legend": { @@ -603,7 +715,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 16, "y": 15 }, + "gridPos": { "h": 8, "w": 8, "x": 16, "y": 23 }, "id": 22, "options": { "legend": { @@ -629,7 +741,90 @@ }, { "collapsed": false, - "gridPos": { "h": 1, "w": 24, "x": 0, "y": 23 }, + "gridPos": { "h": 1, "w": 24, "x": 0, "y": 31 }, + "id": 350, + "title": "Event Lag", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { "mode": "palette-classic" }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { "legend": false, "tooltip": false, "viz": false }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { "type": "linear" }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { "group": "A", "mode": "none" }, + "thresholdsStyle": { "mode": "off" } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { "color": "green", "value": null }, + { "color": "orange", "value": 30 }, + { "color": "red", "value": 120 } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { "h": 8, "w": 24, "x": 0, "y": 32 }, + "id": 25, + "options": { + "legend": { + "calcs": ["lastNotNull"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "sortBy": "Last *", + "sortDesc": true + }, + "tooltip": { "mode": "multi", "sort": "desc" } + }, + "targets": [ + { + "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "editorMode": "code", + "expr": "histogram_quantile(0.50, sum by(event_name, le) (rate(xatu_consumoor_event_lag_seconds_bucket[$__rate_interval])))", + "legendFormat": "p50 {{event_name}}", + "range": true, + "refId": "A" + }, + { + "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "editorMode": "code", + "expr": "histogram_quantile(0.99, sum by(event_name, le) (rate(xatu_consumoor_event_lag_seconds_bucket[$__rate_interval])))", + "legendFormat": "p99 {{event_name}}", + "range": true, + "refId": "B" + } + ], + "title": "Event Lag (p50 / p99)", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { "h": 1, "w": 24, "x": 0, "y": 40 }, "id": 400, "title": "ClickHouse Writes", "type": "row" @@ -674,7 +869,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 0, "y": 24 }, + "gridPos": { "h": 8, "w": 6, "x": 0, "y": 41 }, "id": 30, "options": { "legend": { @@ -742,7 +937,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 8, "y": 24 }, + "gridPos": { "h": 8, "w": 6, "x": 6, "y": 41 }, "id": 31, "options": { "legend": { @@ -816,7 +1011,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 16, "y": 24 }, + "gridPos": { "h": 8, "w": 6, "x": 12, "y": 41 }, "id": 32, "options": { "legend": { @@ -842,9 +1037,77 @@ "title": "Avg Batch Size by Table", "type": "timeseries" }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { "mode": "palette-classic" }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { "legend": false, "tooltip": false, "viz": false }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { "type": "linear" }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { "group": "A", "mode": "none" }, + "thresholdsStyle": { "mode": "off" } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { "color": "green", "value": null }, + { "color": "orange", "value": 0.1 }, + { "color": "red", "value": 1 } + ] + }, + "unit": "ops" + }, + "overrides": [] + }, + "gridPos": { "h": 8, "w": 6, "x": 18, "y": 41 }, + "id": 33, + "options": { + "legend": { + "calcs": ["lastNotNull"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "sortBy": "Last *", + "sortDesc": true + }, + "tooltip": { "mode": "multi", "sort": "desc" } + }, + "targets": [ + { + "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "editorMode": "code", + "expr": "sum by(operation) (rate(xatu_consumoor_write_retries_total[$__rate_interval]))", + "legendFormat": "{{operation}}", + "range": true, + "refId": "A" + } + ], + "title": "Write Retries", + "type": "timeseries" + }, { "collapsed": false, - "gridPos": { "h": 1, "w": 24, "x": 0, "y": 32 }, + "gridPos": { "h": 1, "w": 24, "x": 0, "y": 49 }, "id": 500, "title": "Buffer & Backpressure", "type": "row" @@ -865,8 +1128,8 @@ "axisPlacement": "auto", "barAlignment": 0, "drawStyle": "line", - "fillOpacity": 20, - "gradientMode": "scheme", + "fillOpacity": 10, + "gradientMode": "none", "hideFrom": { "legend": false, "tooltip": false, "viz": false }, "insertNulls": false, "lineInterpolation": "linear", @@ -882,22 +1145,219 @@ "thresholds": { "mode": "absolute", "steps": [ - { "color": "green", "value": null }, - { "color": "orange", "value": 50000 }, - { "color": "red", "value": 100000 } + { "color": "green", "value": null } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { "h": 8, "w": 6, "x": 0, "y": 50 }, + "id": 41, + "options": { + "legend": { + "calcs": ["lastNotNull"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "sortBy": "Last *", + "sortDesc": true + }, + "tooltip": { "mode": "multi", "sort": "desc" } + }, + "targets": [ + { + "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "editorMode": "code", + "expr": "xatu_consumoor_adaptive_limiter_limit", + "legendFormat": "{{table}}", + "range": true, + "refId": "A" + } + ], + "title": "Adaptive Limiter Limit", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { "mode": "palette-classic" }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { "legend": false, "tooltip": false, "viz": false }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { "type": "linear" }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { "group": "A", "mode": "none" }, + "thresholdsStyle": { "mode": "off" } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { "color": "green", "value": null } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { "h": 8, "w": 6, "x": 6, "y": 50 }, + "id": 42, + "options": { + "legend": { + "calcs": ["lastNotNull"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "sortBy": "Last *", + "sortDesc": true + }, + "tooltip": { "mode": "multi", "sort": "desc" } + }, + "targets": [ + { + "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "editorMode": "code", + "expr": "xatu_consumoor_adaptive_limiter_inflight", + "legendFormat": "{{table}}", + "range": true, + "refId": "A" + } + ], + "title": "Adaptive Limiter In-Flight", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { "mode": "palette-classic" }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { "legend": false, "tooltip": false, "viz": false }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { "type": "linear" }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { "group": "A", "mode": "none" }, + "thresholdsStyle": { "mode": "off" } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { "color": "green", "value": null } ] }, "unit": "short" }, "overrides": [] }, - "gridPos": { "h": 8, "w": 24, "x": 0, "y": 33 }, - "id": 40, + "gridPos": { "h": 8, "w": 6, "x": 12, "y": 50 }, + "id": 43, "options": { "legend": { - "calcs": ["lastNotNull", "max"], + "calcs": ["lastNotNull"], "displayMode": "table", - "placement": "right", + "placement": "bottom", + "showLegend": true, + "sortBy": "Last *", + "sortDesc": true + }, + "tooltip": { "mode": "multi", "sort": "desc" } + }, + "targets": [ + { + "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "editorMode": "code", + "expr": "xatu_consumoor_adaptive_limiter_queued", + "legendFormat": "{{table}}", + "range": true, + "refId": "A" + } + ], + "title": "Adaptive Limiter Queued", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { "mode": "palette-classic" }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { "legend": false, "tooltip": false, "viz": false }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { "type": "linear" }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { "group": "A", "mode": "none" }, + "thresholdsStyle": { "mode": "off" } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { "color": "green", "value": null }, + { "color": "red", "value": 1 } + ] + }, + "unit": "ops" + }, + "overrides": [] + }, + "gridPos": { "h": 8, "w": 6, "x": 18, "y": 50 }, + "id": 44, + "options": { + "legend": { + "calcs": ["lastNotNull"], + "displayMode": "table", + "placement": "bottom", "showLegend": true, "sortBy": "Last *", "sortDesc": true @@ -908,18 +1368,18 @@ { "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, "editorMode": "code", - "expr": "xatu_consumoor_buffer_usage", + "expr": "sum by(table) (rate(xatu_consumoor_adaptive_limiter_rejections_total[$__rate_interval]))", "legendFormat": "{{table}}", "range": true, "refId": "A" } ], - "title": "Buffer Usage by Table", + "title": "Adaptive Limiter Rejections", "type": "timeseries" }, { "collapsed": false, - "gridPos": { "h": 1, "w": 24, "x": 0, "y": 41 }, + "gridPos": { "h": 1, "w": 24, "x": 0, "y": 58 }, "id": 600, "title": "Errors", "type": "row" @@ -965,7 +1425,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 6, "x": 0, "y": 42 }, + "gridPos": { "h": 8, "w": 6, "x": 0, "y": 59 }, "id": 50, "options": { "legend": { @@ -1030,7 +1490,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 6, "x": 6, "y": 42 }, + "gridPos": { "h": 8, "w": 6, "x": 6, "y": 59 }, "id": 51, "options": { "legend": { @@ -1095,7 +1555,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 6, "x": 12, "y": 42 }, + "gridPos": { "h": 8, "w": 6, "x": 12, "y": 59 }, "id": 52, "options": { "legend": { @@ -1160,7 +1620,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 6, "x": 18, "y": 42 }, + "gridPos": { "h": 8, "w": 6, "x": 18, "y": 59 }, "id": 53, "options": { "legend": { @@ -1186,7 +1646,7 @@ }, { "collapsed": false, - "gridPos": { "h": 1, "w": 24, "x": 0, "y": 50 }, + "gridPos": { "h": 1, "w": 24, "x": 0, "y": 67 }, "id": 700, "title": "Connection Pool (ch-go)", "type": "row" @@ -1231,7 +1691,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 0, "y": 51 }, + "gridPos": { "h": 8, "w": 8, "x": 0, "y": 68 }, "id": 60, "options": { "legend": { @@ -1327,7 +1787,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 8, "y": 51 }, + "gridPos": { "h": 8, "w": 8, "x": 8, "y": 68 }, "id": 61, "options": { "legend": { @@ -1399,7 +1859,7 @@ }, "overrides": [] }, - "gridPos": { "h": 8, "w": 8, "x": 16, "y": 51 }, + "gridPos": { "h": 8, "w": 8, "x": 16, "y": 68 }, "id": 62, "options": { "legend": { diff --git a/pkg/consumoor/clickhouse/table_writer.go b/pkg/consumoor/clickhouse/table_writer.go index c0b23233..e19fc14e 100644 --- a/pkg/consumoor/clickhouse/table_writer.go +++ b/pkg/consumoor/clickhouse/table_writer.go @@ -190,6 +190,11 @@ func (tw *chTableWriter) do( return poolFn(attemptCtx) } - return tw.limiter.doWithLimiter(attemptCtx, poolFn) + err := tw.limiter.doWithLimiter(attemptCtx, poolFn) + if IsLimiterRejected(err) { + tw.metrics.AdaptiveLimiterRejections().WithLabelValues(tw.table).Inc() + } + + return err }) } diff --git a/pkg/consumoor/clickhouse/writer.go b/pkg/consumoor/clickhouse/writer.go index 08f090bd..3a2fe827 100644 --- a/pkg/consumoor/clickhouse/writer.go +++ b/pkg/consumoor/clickhouse/writer.go @@ -316,6 +316,8 @@ func (w *ChGoWriter) doWithRetry( for attempt := 0; attempt <= w.chgoCfg.MaxRetries; attempt++ { if attempt > 0 { + w.metrics.WriteRetries().WithLabelValues(operation).Inc() + delay := min( w.chgoCfg.RetryBaseDelay*time.Duration(1<<(attempt-1)), w.chgoCfg.RetryMaxDelay, diff --git a/pkg/consumoor/source/benthos.go b/pkg/consumoor/source/benthos.go index 18ed5bd2..ed2d3e54 100644 --- a/pkg/consumoor/source/benthos.go +++ b/pkg/consumoor/source/benthos.go @@ -62,13 +62,14 @@ func NewBenthosStream( service.NewConfigSpec(), func(_ *service.ParsedConfig, _ *service.Resources) (out service.BatchOutput, policy service.BatchPolicy, maxInFlight int, err error) { return &xatuClickHouseOutput{ - log: log.WithField("component", "benthos_clickhouse_output"), - encoding: kafkaConfig.Encoding, - router: routeEngine, - writer: writer, - metrics: metrics, - rejectSink: rejectSink, - ownsWriter: ownsWriter, + log: log.WithField("component", "benthos_clickhouse_output"), + encoding: kafkaConfig.Encoding, + router: routeEngine, + writer: writer, + metrics: metrics, + rejectSink: rejectSink, + ownsWriter: ownsWriter, + outputBatchCount: kafkaConfig.OutputBatchCount, }, batchPolicy, kafkaConfig.MaxInFlight, nil }, ); registerErr != nil { diff --git a/pkg/consumoor/source/output.go b/pkg/consumoor/source/output.go index 1cce7a78..d0651e01 100644 --- a/pkg/consumoor/source/output.go +++ b/pkg/consumoor/source/output.go @@ -38,13 +38,14 @@ type eventGroup struct { } type xatuClickHouseOutput struct { - log logrus.FieldLogger - encoding string - router *router.Engine - writer Writer - metrics *telemetry.Metrics - rejectSink rejectSink - ownsWriter bool + log logrus.FieldLogger + encoding string + router *router.Engine + writer Writer + metrics *telemetry.Metrics + rejectSink rejectSink + ownsWriter bool + outputBatchCount int mu sync.Mutex started bool @@ -83,6 +84,18 @@ func (o *xatuClickHouseOutput) WriteBatch( return ctx.Err() } + // Record which trigger caused this batch flush. + // Each Benthos stream is per-topic, so the first message's topic is + // representative of the entire batch. + topic := kafkaMetadata(msgs[0]).Topic + + trigger := "timeout" + if o.outputBatchCount > 0 && len(msgs) >= o.outputBatchCount { + trigger = "count" + } + + o.metrics.BatchFlushTrigger().WithLabelValues(topic, trigger).Inc() + var batchErr *service.BatchError var pooledEvents []*xatu.DecoratedEvent diff --git a/pkg/consumoor/telemetry/metrics.go b/pkg/consumoor/telemetry/metrics.go index adef6d99..76ebd017 100644 --- a/pkg/consumoor/telemetry/metrics.go +++ b/pkg/consumoor/telemetry/metrics.go @@ -20,14 +20,17 @@ type Metrics struct { batchSize *prometheus.HistogramVec flattenErrors *prometheus.CounterVec eventLag *prometheus.HistogramVec + batchFlushTrigger *prometheus.CounterVec activeTopics prometheus.Gauge kafkaConsumerLag *prometheus.GaugeVec outputMaxInFlight prometheus.Gauge + writeRetries *prometheus.CounterVec // adaptive limiter metrics (per-table) - adaptiveLimiterLimit *prometheus.GaugeVec - adaptiveLimiterInflight *prometheus.GaugeVec - adaptiveLimiterQueued *prometheus.GaugeVec + adaptiveLimiterLimit *prometheus.GaugeVec + adaptiveLimiterInflight *prometheus.GaugeVec + adaptiveLimiterQueued *prometheus.GaugeVec + adaptiveLimiterRejections *prometheus.CounterVec // ch-go pool metrics chgoPoolAcquiredResources prometheus.Gauge @@ -145,6 +148,13 @@ func NewMetrics(namespace string) *Metrics { Buckets: []float64{0.1, 0.5, 1, 2, 5, 10, 30, 60, 120, 300, 600}, }, []string{"event_name"}), + batchFlushTrigger: promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "batch_flush_trigger_total", + Help: "Total number of batch flushes by trigger type (count or timeout).", + }, []string{"topic", "trigger"}), + activeTopics: promauto.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, @@ -187,6 +197,20 @@ func NewMetrics(namespace string) *Metrics { Help: "Current number of queued permit requests per table.", }, []string{"table"}), + adaptiveLimiterRejections: promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "adaptive_limiter_rejections_total", + Help: "Total number of requests rejected by the adaptive concurrency limiter.", + }, []string{"table"}), + + writeRetries: promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "write_retries_total", + Help: "Total number of ch-go operation retry attempts after transient errors.", + }, []string{"operation"}), + chgoPoolAcquiredResources: promauto.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, @@ -259,26 +283,31 @@ func NewMetrics(namespace string) *Metrics { } } -func (m *Metrics) MessagesConsumed() *prometheus.CounterVec { return m.messagesConsumed } -func (m *Metrics) MessagesRouted() *prometheus.CounterVec { return m.messagesRouted } -func (m *Metrics) MessagesDropped() *prometheus.CounterVec { return m.messagesDropped } -func (m *Metrics) MessagesRejected() *prometheus.CounterVec { return m.messagesRejected } -func (m *Metrics) DecodeErrors() *prometheus.CounterVec { return m.decodeErrors } -func (m *Metrics) DLQWrites() *prometheus.CounterVec { return m.dlqWrites } -func (m *Metrics) DLQErrors() *prometheus.CounterVec { return m.dlqErrors } -func (m *Metrics) RowsWritten() *prometheus.CounterVec { return m.rowsWritten } -func (m *Metrics) WriteErrors() *prometheus.CounterVec { return m.writeErrors } -func (m *Metrics) WriteDuration() *prometheus.HistogramVec { return m.writeDuration } -func (m *Metrics) BatchSize() *prometheus.HistogramVec { return m.batchSize } -func (m *Metrics) FlattenErrors() *prometheus.CounterVec { return m.flattenErrors } -func (m *Metrics) EventLag() *prometheus.HistogramVec { return m.eventLag } -func (m *Metrics) ActiveTopics() prometheus.Gauge { return m.activeTopics } -func (m *Metrics) KafkaConsumerLag() *prometheus.GaugeVec { return m.kafkaConsumerLag } -func (m *Metrics) OutputMaxInFlight() prometheus.Gauge { return m.outputMaxInFlight } +func (m *Metrics) MessagesConsumed() *prometheus.CounterVec { return m.messagesConsumed } +func (m *Metrics) MessagesRouted() *prometheus.CounterVec { return m.messagesRouted } +func (m *Metrics) MessagesDropped() *prometheus.CounterVec { return m.messagesDropped } +func (m *Metrics) MessagesRejected() *prometheus.CounterVec { return m.messagesRejected } +func (m *Metrics) DecodeErrors() *prometheus.CounterVec { return m.decodeErrors } +func (m *Metrics) DLQWrites() *prometheus.CounterVec { return m.dlqWrites } +func (m *Metrics) DLQErrors() *prometheus.CounterVec { return m.dlqErrors } +func (m *Metrics) RowsWritten() *prometheus.CounterVec { return m.rowsWritten } +func (m *Metrics) WriteErrors() *prometheus.CounterVec { return m.writeErrors } +func (m *Metrics) WriteDuration() *prometheus.HistogramVec { return m.writeDuration } +func (m *Metrics) BatchSize() *prometheus.HistogramVec { return m.batchSize } +func (m *Metrics) FlattenErrors() *prometheus.CounterVec { return m.flattenErrors } +func (m *Metrics) EventLag() *prometheus.HistogramVec { return m.eventLag } +func (m *Metrics) BatchFlushTrigger() *prometheus.CounterVec { return m.batchFlushTrigger } +func (m *Metrics) ActiveTopics() prometheus.Gauge { return m.activeTopics } +func (m *Metrics) KafkaConsumerLag() *prometheus.GaugeVec { return m.kafkaConsumerLag } +func (m *Metrics) OutputMaxInFlight() prometheus.Gauge { return m.outputMaxInFlight } func (m *Metrics) AdaptiveLimiterLimit() *prometheus.GaugeVec { return m.adaptiveLimiterLimit } func (m *Metrics) AdaptiveLimiterInflight() *prometheus.GaugeVec { return m.adaptiveLimiterInflight } func (m *Metrics) AdaptiveLimiterQueued() *prometheus.GaugeVec { return m.adaptiveLimiterQueued } +func (m *Metrics) AdaptiveLimiterRejections() *prometheus.CounterVec { + return m.adaptiveLimiterRejections +} +func (m *Metrics) WriteRetries() *prometheus.CounterVec { return m.writeRetries } func (m *Metrics) ChgoPoolAcquiredResources() prometheus.Gauge { return m.chgoPoolAcquiredResources } func (m *Metrics) ChgoPoolIdleResources() prometheus.Gauge { return m.chgoPoolIdleResources } From 0a503592230bec215e82e01cb7e1dae49e1ee2d0 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Fri, 27 Feb 2026 12:43:19 +1000 Subject: [PATCH 16/27] fix(consumoor): align data transformations with Vector pipeline - libp2p_rpc_data_column_custody_probe: compute peer_id_unique_key from metadata peer_id only (no payload fallback), matching Vector behavior - libp2p_rpc_meta_control_prune: write NULL for graft_peer_id_unique_key when no graft peer instead of seahash(networkName) - beacon chain_reorg: source epoch from additional_data instead of payload - canonical_beacon_blob_sidecar: source slot from additional_data - canonical_beacon_committee: source slot from payload - canonical_beacon_elaborated_attestation: source slot from payload - meta_client_ip: write NULL instead of :: when IP is empty/unparseable across all 77 generated route files, with new NormalizeIPToIPv6Nullable helper and updated code generator --- .../beacon_api_eth_v1_beacon_blob.gen.go | 2 +- .../beacon_api_eth_v1_beacon_committee.gen.go | 2 +- ...eacon_api_eth_v1_events_attestation.gen.go | 2 +- ...acon_api_eth_v1_events_blob_sidecar.gen.go | 2 +- .../beacon_api_eth_v1_events_block.gen.go | 2 +- ...acon_api_eth_v1_events_block_gossip.gen.go | 2 +- ...eacon_api_eth_v1_events_chain_reorg.gen.go | 2 +- .../beacon_api_eth_v1_events_chain_reorg.go | 13 +----- ...th_v1_events_contribution_and_proof.gen.go | 2 +- ...i_eth_v1_events_data_column_sidecar.gen.go | 2 +- ..._eth_v1_events_finalized_checkpoint.gen.go | 2 +- .../beacon_api_eth_v1_events_head.gen.go | 2 +- ...on_api_eth_v1_events_voluntary_exit.gen.go | 2 +- .../beacon_api_eth_v1_proposer_duty.gen.go | 2 +- ...i_eth_v1_validator_attestation_data.gen.go | 2 +- .../beacon_api_eth_v2_beacon_block.gen.go | 2 +- .../beacon_api_eth_v3_validator_block.gen.go | 2 +- .../canonical_beacon_blob_sidecar.gen.go | 2 +- .../canonical_beacon_blob_sidecar.go | 35 ++++++++-------- .../canonical_beacon_blob_sidecar_test.go | 1 + .../canonical/canonical_beacon_block.gen.go | 2 +- ...ical_beacon_block_attester_slashing.gen.go | 2 +- ...eacon_block_bls_to_execution_change.gen.go | 2 +- .../canonical_beacon_block_deposit.gen.go | 2 +- ..._beacon_block_execution_transaction.gen.go | 2 +- ...ical_beacon_block_proposer_slashing.gen.go | 2 +- ...nonical_beacon_block_sync_aggregate.gen.go | 2 +- ...nonical_beacon_block_voluntary_exit.gen.go | 2 +- .../canonical_beacon_block_withdrawal.gen.go | 2 +- .../canonical_beacon_committee.gen.go | 2 +- .../canonical/canonical_beacon_committee.go | 35 ++++++++-------- ...nical_beacon_elaborated_attestation.gen.go | 2 +- ...canonical_beacon_elaborated_attestation.go | 41 ++++++++++--------- .../canonical_beacon_proposer_duty.gen.go | 2 +- .../canonical_beacon_sync_committee.gen.go | 2 +- .../canonical_beacon_validators.gen.go | 2 +- ...canonical_beacon_validators_pubkeys.gen.go | 2 +- ...n_validators_withdrawal_credentials.gen.go | 2 +- pkg/consumoor/route/cmd/chgo-rowgen/main.go | 11 ++++- pkg/consumoor/route/column.go | 20 +++++++++ .../consensus_engine_api_get_blobs.gen.go | 2 +- .../consensus_engine_api_new_payload.gen.go | 2 +- .../execution/execution_block_metrics.gen.go | 2 +- .../execution_engine_get_blobs.gen.go | 2 +- .../execution_engine_new_payload.gen.go | 2 +- .../execution/execution_state_size.gen.go | 2 +- .../execution/mempool_transaction.gen.go | 2 +- .../route/libp2p/libp2p_add_peer.gen.go | 2 +- .../route/libp2p/libp2p_connected.gen.go | 2 +- .../libp2p/libp2p_deliver_message.gen.go | 2 +- .../route/libp2p/libp2p_disconnected.gen.go | 2 +- .../route/libp2p/libp2p_drop_rpc.gen.go | 2 +- .../libp2p/libp2p_duplicate_message.gen.go | 2 +- ...ibp2p_gossipsub_aggregate_and_proof.gen.go | 2 +- ...libp2p_gossipsub_beacon_attestation.gen.go | 2 +- .../libp2p_gossipsub_beacon_block.gen.go | 2 +- .../libp2p_gossipsub_blob_sidecar.gen.go | 2 +- ...ibp2p_gossipsub_data_column_sidecar.gen.go | 2 +- .../route/libp2p/libp2p_graft.gen.go | 2 +- .../libp2p/libp2p_handle_metadata.gen.go | 2 +- .../route/libp2p/libp2p_handle_status.gen.go | 2 +- .../route/libp2p/libp2p_identify.gen.go | 2 +- pkg/consumoor/route/libp2p/libp2p_join.gen.go | 2 +- .../route/libp2p/libp2p_leave.gen.go | 2 +- .../route/libp2p/libp2p_prune.gen.go | 2 +- .../libp2p/libp2p_publish_message.gen.go | 2 +- .../route/libp2p/libp2p_recv_rpc.gen.go | 2 +- .../route/libp2p/libp2p_reject_message.gen.go | 2 +- .../route/libp2p/libp2p_remove_peer.gen.go | 2 +- ...ibp2p_rpc_data_column_custody_probe.gen.go | 2 +- .../libp2p_rpc_data_column_custody_probe.go | 12 +++--- ...bp2p_rpc_data_column_custody_probe_test.go | 6 ++- .../libp2p_rpc_meta_control_graft.gen.go | 2 +- .../libp2p_rpc_meta_control_idontwant.gen.go | 2 +- .../libp2p_rpc_meta_control_ihave.gen.go | 2 +- .../libp2p_rpc_meta_control_iwant.gen.go | 2 +- .../libp2p_rpc_meta_control_prune.gen.go | 2 +- .../libp2p/libp2p_rpc_meta_control_prune.go | 4 +- .../libp2p/libp2p_rpc_meta_message.gen.go | 2 +- .../libp2p_rpc_meta_subscription.gen.go | 2 +- .../route/libp2p/libp2p_send_rpc.gen.go | 2 +- .../libp2p/libp2p_synthetic_heartbeat.gen.go | 2 +- .../route/mev/mev_relay_bid_trace.gen.go | 2 +- ...ev_relay_proposer_payload_delivered.gen.go | 2 +- .../mev_relay_validator_registration.gen.go | 2 +- .../route/node/node_record_consensus.gen.go | 2 +- .../route/node/node_record_execution.gen.go | 2 +- 87 files changed, 180 insertions(+), 152 deletions(-) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go index 25c9b9a2..26e65126 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go @@ -101,7 +101,7 @@ func (b *beaconApiEthV1BeaconBlobBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go index 6d0deed4..9ec80ba1 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go @@ -95,7 +95,7 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) appendMetadata(event *xatu.Decorate b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go index 5e491abf..6c6411e6 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go @@ -107,7 +107,7 @@ func (b *beaconApiEthV1EventsAttestationBatch) appendMetadata(event *xatu.Decora b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go index 742c1bc2..ce023fe4 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go @@ -100,7 +100,7 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) appendMetadata(event *xatu.Decora b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go index 5c4c30af..68e3a1ac 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go @@ -96,7 +96,7 @@ func (b *beaconApiEthV1EventsBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go index 1217a15e..c2a71ea5 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go @@ -95,7 +95,7 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) appendMetadata(event *xatu.Decora b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go index 68cc31dc..bf5cf27c 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go @@ -103,7 +103,7 @@ func (b *beaconApiEthV1EventsChainReorgBatch) appendMetadata(event *xatu.Decorat b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.go index e5ffd27f..d9e7dc44 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.go @@ -64,10 +64,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) validate(event *xatu.DecoratedEven return fmt.Errorf("nil Depth: %w", route.ErrInvalidEvent) } - if payload.GetEpoch() == nil { - return fmt.Errorf("nil Epoch: %w", route.ErrInvalidEvent) - } - return nil } @@ -99,13 +95,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) appendPayload(event *xatu.Decorate b.NewHeadBlock.Append([]byte(chainReorgV2.GetNewHeadBlock())) b.OldHeadState.Append([]byte(chainReorgV2.GetOldHeadState())) b.NewHeadState.Append([]byte(chainReorgV2.GetNewHeadState())) - - if epoch := chainReorgV2.GetEpoch(); epoch != nil { - b.Epoch.Append(uint32(epoch.GetValue())) //nolint:gosec // epoch fits uint32 - } else { - b.Epoch.Append(0) - } - b.ExecutionOptimistic.Append(false) } @@ -113,6 +102,7 @@ func (b *beaconApiEthV1EventsChainReorgBatch) appendAdditionalData(event *xatu.D if event.GetMeta() == nil || event.GetMeta().GetClient() == nil { b.SlotStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) + b.Epoch.Append(0) b.EpochStartDateTime.Append(time.Time{}) return @@ -123,5 +113,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) appendAdditionalData(event *xatu.D b.SlotStartDateTime.Append(time.Unix(additional.SlotStartDateTime, 0)) b.PropagationSlotStartDiff.Append(uint32(additional.PropagationSlotStartDiff)) //nolint:gosec // propagation diff fits uint32 + b.Epoch.Append(uint32(additional.Epoch)) //nolint:gosec // epoch fits uint32 b.EpochStartDateTime.Append(time.Unix(additional.EpochStartDateTime, 0)) } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go index 78ee3af5..640f147d 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go @@ -101,7 +101,7 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) appendMetadata(event *xa b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go index dd1e3412..e70b399a 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go @@ -97,7 +97,7 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendMetadata(event *xatu. b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go index 13d428d0..b40e5adc 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go @@ -95,7 +95,7 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) appendMetadata(event *xat b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go index 2f5357d8..03901ab4 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go @@ -101,7 +101,7 @@ func (b *beaconApiEthV1EventsHeadBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go index c924dda4..0dbfd352 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go @@ -96,7 +96,7 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) appendMetadata(event *xatu.Deco b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go index 8f3f2be4..ba8e9026 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go @@ -94,7 +94,7 @@ func (b *beaconApiEthV1ProposerDutyBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go index b8ddf4db..e69e44bf 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go @@ -106,7 +106,7 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) appendMetadata(event *xatu b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go index 02b4250b..09bc3459 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go @@ -132,7 +132,7 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go index 48749a6f..eea9f1c3 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go @@ -118,7 +118,7 @@ func (b *beaconApiEthV3ValidatorBlockBatch) appendMetadata(event *xatu.Decorated b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go index 5cf5a719..dd471da7 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go @@ -105,7 +105,7 @@ func (b *canonicalBeaconBlobSidecarBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.go b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.go index 763af395..d59b7421 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.go @@ -76,11 +76,6 @@ func (b *canonicalBeaconBlobSidecarBatch) appendRuntime(_ *xatu.DecoratedEvent) func (b *canonicalBeaconBlobSidecarBatch) appendPayload(event *xatu.DecoratedEvent) { blob := event.GetEthV1BeaconBlockBlobSidecar() - if slot := blob.GetSlot(); slot != nil { - b.Slot.Append(uint32(slot.GetValue())) //nolint:gosec // G115 - } else { - b.Slot.Append(0) - } b.BlockRoot.Append([]byte(blob.GetBlockRoot())) b.BlockParentRoot.Append([]byte(blob.GetBlockParentRoot())) @@ -104,9 +99,10 @@ func (b *canonicalBeaconBlobSidecarBatch) appendPayload(event *xatu.DecoratedEve func (b *canonicalBeaconBlobSidecarBatch) appendAdditionalData(event *xatu.DecoratedEvent) { additional := event.GetMeta().GetClient().GetEthV1BeaconBlobSidecar() if additional == nil { + b.Slot.Append(0) + b.SlotStartDateTime.Append(time.Time{}) b.Epoch.Append(0) b.EpochStartDateTime.Append(time.Time{}) - b.SlotStartDateTime.Append(time.Time{}) b.VersionedHash.Append(nil) b.BlobSize.Append(0) b.BlobEmptySize.Append(proto.Nullable[uint32]{}) @@ -114,6 +110,23 @@ func (b *canonicalBeaconBlobSidecarBatch) appendAdditionalData(event *xatu.Decor return } + if slot := additional.GetSlot(); slot != nil { + if slotNumber := slot.GetNumber(); slotNumber != nil { + b.Slot.Append(uint32(slotNumber.GetValue())) //nolint:gosec // G115 + } else { + b.Slot.Append(0) + } + + if startDateTime := slot.GetStartDateTime(); startDateTime != nil { + b.SlotStartDateTime.Append(startDateTime.AsTime()) + } else { + b.SlotStartDateTime.Append(time.Time{}) + } + } else { + b.Slot.Append(0) + b.SlotStartDateTime.Append(time.Time{}) + } + if epoch := additional.GetEpoch(); epoch != nil { if epochNumber := epoch.GetNumber(); epochNumber != nil { b.Epoch.Append(uint32(epochNumber.GetValue())) //nolint:gosec // G115 @@ -131,16 +144,6 @@ func (b *canonicalBeaconBlobSidecarBatch) appendAdditionalData(event *xatu.Decor b.EpochStartDateTime.Append(time.Time{}) } - if slot := additional.GetSlot(); slot != nil { - if startDateTime := slot.GetStartDateTime(); startDateTime != nil { - b.SlotStartDateTime.Append(startDateTime.AsTime()) - } else { - b.SlotStartDateTime.Append(time.Time{}) - } - } else { - b.SlotStartDateTime.Append(time.Time{}) - } - b.VersionedHash.Append([]byte(additional.GetVersionedHash())) if dataSize := additional.GetDataSize(); dataSize != nil { diff --git a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar_test.go b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar_test.go index 52a166bc..f89f1d56 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar_test.go @@ -19,6 +19,7 @@ func TestSnapshot_canonical_beacon_blob_sidecar(t *testing.T) { Meta: testfixture.MetaWithAdditional(&xatu.ClientMeta{ AdditionalData: &xatu.ClientMeta_EthV1BeaconBlobSidecar{ EthV1BeaconBlobSidecar: &xatu.ClientMeta_AdditionalEthV1BeaconBlobSidecarData{ + Slot: testfixture.SlotEpochAdditional(), Epoch: testfixture.EpochAdditional(), }, }, diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go index f4c47105..d145f5a5 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go @@ -132,7 +132,7 @@ func (b *canonicalBeaconBlockBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go index 4919c8a0..81e2ef4d 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go @@ -119,7 +119,7 @@ func (b *canonicalBeaconBlockAttesterSlashingBatch) appendMetadata(event *xatu.D b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go index f9edb776..cad963a7 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go @@ -98,7 +98,7 @@ func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) appendMetadata(event *xa b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go index 043fe41c..cffd4cc3 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go @@ -100,7 +100,7 @@ func (b *canonicalBeaconBlockDepositBatch) appendMetadata(event *xatu.DecoratedE b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go index 8931c837..2178a773 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go @@ -121,7 +121,7 @@ func (b *canonicalBeaconBlockExecutionTransactionBatch) appendMetadata(event *xa b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go index 9c188622..f3bc9d3e 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go @@ -111,7 +111,7 @@ func (b *canonicalBeaconBlockProposerSlashingBatch) appendMetadata(event *xatu.D b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go index d694eebd..35c1436a 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go @@ -103,7 +103,7 @@ func (b *canonicalBeaconBlockSyncAggregateBatch) appendMetadata(event *xatu.Deco b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go index 563c937c..67b6e6f5 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go @@ -96,7 +96,7 @@ func (b *canonicalBeaconBlockVoluntaryExitBatch) appendMetadata(event *xatu.Deco b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go index 13424507..c4a01ff6 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go @@ -98,7 +98,7 @@ func (b *canonicalBeaconBlockWithdrawalBatch) appendMetadata(event *xatu.Decorat b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go index b3651dee..d5e7fd46 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go @@ -93,7 +93,7 @@ func (b *canonicalBeaconCommitteeBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_committee.go b/pkg/consumoor/route/canonical/canonical_beacon_committee.go index 379d78a1..31b5344d 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_committee.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_committee.go @@ -66,6 +66,13 @@ func (b *canonicalBeaconCommitteeBatch) appendRuntime(_ *xatu.DecoratedEvent) { func (b *canonicalBeaconCommitteeBatch) appendPayload(event *xatu.DecoratedEvent) { committee := event.GetEthV1BeaconCommittee() + + if slot := committee.GetSlot(); slot != nil { + b.Slot.Append(uint32(slot.GetValue())) //nolint:gosec // G115 + } else { + b.Slot.Append(0) + } + if index := committee.GetIndex(); index != nil { b.CommitteeIndex.Append(strconv.FormatUint(index.GetValue(), 10)) } else { @@ -79,7 +86,6 @@ func (b *canonicalBeaconCommitteeBatch) appendPayload(event *xatu.DecoratedEvent func (b *canonicalBeaconCommitteeBatch) appendAdditionalData(event *xatu.DecoratedEvent) { additional := event.GetMeta().GetClient().GetEthV1BeaconCommittee() if additional == nil { - b.Slot.Append(0) b.SlotStartDateTime.Append(time.Time{}) b.Epoch.Append(0) b.EpochStartDateTime.Append(time.Time{}) @@ -87,6 +93,16 @@ func (b *canonicalBeaconCommitteeBatch) appendAdditionalData(event *xatu.Decorat return } + if slotData := additional.GetSlot(); slotData != nil { + if startDateTime := slotData.GetStartDateTime(); startDateTime != nil { + b.SlotStartDateTime.Append(startDateTime.AsTime()) + } else { + b.SlotStartDateTime.Append(time.Time{}) + } + } else { + b.SlotStartDateTime.Append(time.Time{}) + } + if epochData := additional.GetEpoch(); epochData != nil { if epochNumber := epochData.GetNumber(); epochNumber != nil { b.Epoch.Append(uint32(epochNumber.GetValue())) @@ -103,21 +119,4 @@ func (b *canonicalBeaconCommitteeBatch) appendAdditionalData(event *xatu.Decorat b.Epoch.Append(0) b.EpochStartDateTime.Append(time.Time{}) } - - if slotData := additional.GetSlot(); slotData != nil { - if slotNumber := slotData.GetNumber(); slotNumber != nil { - b.Slot.Append(uint32(slotNumber.GetValue())) - } else { - b.Slot.Append(0) - } - - if startDateTime := slotData.GetStartDateTime(); startDateTime != nil { - b.SlotStartDateTime.Append(startDateTime.AsTime()) - } else { - b.SlotStartDateTime.Append(time.Time{}) - } - } else { - b.Slot.Append(0) - b.SlotStartDateTime.Append(time.Time{}) - } } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go index 44986dad..2abaca7a 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go @@ -110,7 +110,7 @@ func (b *canonicalBeaconElaboratedAttestationBatch) appendMetadata(event *xatu.D b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.go b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.go index 213a6df9..38beb0cb 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.go @@ -109,7 +109,18 @@ func (b *canonicalBeaconElaboratedAttestationBatch) appendRow(event *xatu.Decora b.TargetRoot.Append(nil) } - // Additional data fields (including Slot which overrides the payload slot). + // Slot from payload (matching Vector: .slot = .data.data.slot). + if attestation != nil && attestation.GetData() != nil { + if slot := attestation.GetData().GetSlot(); slot != nil { + b.Slot.Append(uint32(slot.GetValue())) + } else { + b.Slot.Append(0) + } + } else { + b.Slot.Append(0) + } + + // Additional data fields. additional := event.GetMeta().GetClient().GetEthV2BeaconBlockElaboratedAttestation() if additional == nil { b.BlockSlot.Append(0) @@ -118,7 +129,6 @@ func (b *canonicalBeaconElaboratedAttestationBatch) appendRow(event *xatu.Decora b.BlockEpochStartDateTime.Append(time.Time{}) b.BlockRoot.Append(nil) b.PositionInBlock.Append(0) - b.Slot.Append(0) b.SlotStartDateTime.Append(time.Time{}) b.Epoch.Append(0) b.EpochStartDateTime.Append(time.Time{}) @@ -137,6 +147,16 @@ func (b *canonicalBeaconElaboratedAttestationBatch) appendRow(event *xatu.Decora b.PositionInBlock.Append(0) } + if slotData := additional.GetSlot(); slotData != nil { + if startDateTime := slotData.GetStartDateTime(); startDateTime != nil { + b.SlotStartDateTime.Append(startDateTime.AsTime()) + } else { + b.SlotStartDateTime.Append(time.Time{}) + } + } else { + b.SlotStartDateTime.Append(time.Time{}) + } + if epochData := additional.GetEpoch(); epochData != nil { if epochNumber := epochData.GetNumber(); epochNumber != nil { b.Epoch.Append(uint32(epochNumber.GetValue())) @@ -154,23 +174,6 @@ func (b *canonicalBeaconElaboratedAttestationBatch) appendRow(event *xatu.Decora b.EpochStartDateTime.Append(time.Time{}) } - if slotData := additional.GetSlot(); slotData != nil { - if slotNumber := slotData.GetNumber(); slotNumber != nil { - b.Slot.Append(uint32(slotNumber.GetValue())) - } else { - b.Slot.Append(0) - } - - if startDateTime := slotData.GetStartDateTime(); startDateTime != nil { - b.SlotStartDateTime.Append(startDateTime.AsTime()) - } else { - b.SlotStartDateTime.Append(time.Time{}) - } - } else { - b.Slot.Append(0) - b.SlotStartDateTime.Append(time.Time{}) - } - if source := additional.GetSource(); source != nil { if sourceEpoch := source.GetEpoch(); sourceEpoch != nil { if startDateTime := sourceEpoch.GetStartDateTime(); startDateTime != nil { diff --git a/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go index ac2c8510..7417cbb0 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go @@ -92,7 +92,7 @@ func (b *canonicalBeaconProposerDutyBatch) appendMetadata(event *xatu.DecoratedE b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go index 80d420db..8133e591 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go @@ -93,7 +93,7 @@ func (b *canonicalBeaconSyncCommitteeBatch) appendMetadata(event *xatu.Decorated b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go index 7765b16c..f271bcdb 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go @@ -103,7 +103,7 @@ func (b *canonicalBeaconValidatorsBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go index 2170bf85..ecaf43c8 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go @@ -91,7 +91,7 @@ func (b *canonicalBeaconValidatorsPubkeysBatch) appendMetadata(event *xatu.Decor b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go index 6943e268..aed0b34d 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go @@ -91,7 +91,7 @@ func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) appendMetadata(eve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/cmd/chgo-rowgen/main.go b/pkg/consumoor/route/cmd/chgo-rowgen/main.go index 9d32a23c..2beebcff 100644 --- a/pkg/consumoor/route/cmd/chgo-rowgen/main.go +++ b/pkg/consumoor/route/cmd/chgo-rowgen/main.go @@ -1005,8 +1005,8 @@ var metaColumnDefs = map[string]metaColumnDef{ "meta_client_clock_drift": {expr: "event.GetMeta().GetClient().GetClockDrift()", goType: goTypeUint64}, "meta_client_module_name": {expr: "event.GetMeta().GetClient().GetModuleName().String()"}, - // Client IP (normalized to IPv6 in a single parse) - "meta_client_ip": {expr: "route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP())", goType: protoIPv6}, + // Client IP (nullable – NULL when empty/unparseable, matching Vector behavior) + "meta_client_ip": {special: "nullable_ip"}, // Client geo "meta_client_geo_city": {expr: "event.GetMeta().GetServer().GetClient().GetGeo().GetCity()"}, @@ -1105,6 +1105,13 @@ func writeAppendMetadata(b *bytes.Buffer, batchName string, cols []column, gens continue } + // Special handling for nullable IP (NULL when empty/unparseable). + if def.special == "nullable_ip" { + fmt.Fprintf(b, "\tb.%s.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP()))\n", col.Field) + + continue + } + // Consensus version columns: emit the single parse call on // first encounter, then reference the local variables. if _, isCv := cvFieldMap[def.special]; isCv { diff --git a/pkg/consumoor/route/column.go b/pkg/consumoor/route/column.go index bd17c2f5..e63049ce 100644 --- a/pkg/consumoor/route/column.go +++ b/pkg/consumoor/route/column.go @@ -32,6 +32,26 @@ func NormalizeIPToIPv6(raw string) proto.IPv6 { return proto.ToIPv6(addr) } +// NormalizeIPToIPv6Nullable parses an IP string and returns a Nullable IPv6. +// Returns NULL when the input is empty or unparseable (matching Vector behavior), +// rather than writing the zero address "::". +func NormalizeIPToIPv6Nullable(raw string) proto.Nullable[proto.IPv6] { + if raw == "" { + return proto.Nullable[proto.IPv6]{} + } + + addr, err := netip.ParseAddr(raw) + if err != nil { + return proto.Nullable[proto.IPv6]{} + } + + if addr.Is4() { + addr = netip.AddrFrom16(addr.As16()) + } + + return proto.NewNullable[proto.IPv6](proto.ToIPv6(addr)) +} + // NormalizeConsensusVersion strips the implementation prefix from a // consensus version string (e.g. "Lighthouse/v4.5.0" -> "v4.5.0"). func NormalizeConsensusVersion(raw string) string { diff --git a/pkg/consumoor/route/execution/consensus_engine_api_get_blobs.gen.go b/pkg/consumoor/route/execution/consensus_engine_api_get_blobs.gen.go index b09a4013..7ca229f8 100644 --- a/pkg/consumoor/route/execution/consensus_engine_api_get_blobs.gen.go +++ b/pkg/consumoor/route/execution/consensus_engine_api_get_blobs.gen.go @@ -113,7 +113,7 @@ func (b *consensusEngineApiGetBlobsBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/execution/consensus_engine_api_new_payload.gen.go b/pkg/consumoor/route/execution/consensus_engine_api_new_payload.gen.go index 93c631e6..aa297d61 100644 --- a/pkg/consumoor/route/execution/consensus_engine_api_new_payload.gen.go +++ b/pkg/consumoor/route/execution/consensus_engine_api_new_payload.gen.go @@ -117,7 +117,7 @@ func (b *consensusEngineApiNewPayloadBatch) appendMetadata(event *xatu.Decorated b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/execution/execution_block_metrics.gen.go b/pkg/consumoor/route/execution/execution_block_metrics.gen.go index a7a04954..ac347530 100644 --- a/pkg/consumoor/route/execution/execution_block_metrics.gen.go +++ b/pkg/consumoor/route/execution/execution_block_metrics.gen.go @@ -111,7 +111,7 @@ func (b *executionBlockMetricsBatch) appendMetadata(event *xatu.DecoratedEvent) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/execution/execution_engine_get_blobs.gen.go b/pkg/consumoor/route/execution/execution_engine_get_blobs.gen.go index 4762cd1c..704da51c 100644 --- a/pkg/consumoor/route/execution/execution_engine_get_blobs.gen.go +++ b/pkg/consumoor/route/execution/execution_engine_get_blobs.gen.go @@ -108,7 +108,7 @@ func (b *executionEngineGetBlobsBatch) appendMetadata(event *xatu.DecoratedEvent b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/execution/execution_engine_new_payload.gen.go b/pkg/consumoor/route/execution/execution_engine_new_payload.gen.go index da77efca..3ed09116 100644 --- a/pkg/consumoor/route/execution/execution_engine_new_payload.gen.go +++ b/pkg/consumoor/route/execution/execution_engine_new_payload.gen.go @@ -109,7 +109,7 @@ func (b *executionEngineNewPayloadBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/execution/execution_state_size.gen.go b/pkg/consumoor/route/execution/execution_state_size.gen.go index bfcbdd73..d01f4a53 100644 --- a/pkg/consumoor/route/execution/execution_state_size.gen.go +++ b/pkg/consumoor/route/execution/execution_state_size.gen.go @@ -101,7 +101,7 @@ func (b *executionStateSizeBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/execution/mempool_transaction.gen.go b/pkg/consumoor/route/execution/mempool_transaction.gen.go index 2ed3095a..e4a98230 100644 --- a/pkg/consumoor/route/execution/mempool_transaction.gen.go +++ b/pkg/consumoor/route/execution/mempool_transaction.gen.go @@ -110,7 +110,7 @@ func (b *mempoolTransactionBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_add_peer.gen.go b/pkg/consumoor/route/libp2p/libp2p_add_peer.gen.go index 2f8052ad..34a195f5 100644 --- a/pkg/consumoor/route/libp2p/libp2p_add_peer.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_add_peer.gen.go @@ -77,7 +77,7 @@ func (b *libp2pAddPeerBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_connected.gen.go b/pkg/consumoor/route/libp2p/libp2p_connected.gen.go index b677c09b..bcd9bb9f 100644 --- a/pkg/consumoor/route/libp2p/libp2p_connected.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_connected.gen.go @@ -102,7 +102,7 @@ func (b *libp2pConnectedBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_deliver_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_deliver_message.gen.go index a761ba3e..23776db1 100644 --- a/pkg/consumoor/route/libp2p/libp2p_deliver_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_deliver_message.gen.go @@ -84,7 +84,7 @@ func (b *libp2pDeliverMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_disconnected.gen.go b/pkg/consumoor/route/libp2p/libp2p_disconnected.gen.go index 28413184..c41ee1c6 100644 --- a/pkg/consumoor/route/libp2p/libp2p_disconnected.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_disconnected.gen.go @@ -102,7 +102,7 @@ func (b *libp2pDisconnectedBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_drop_rpc.gen.go b/pkg/consumoor/route/libp2p/libp2p_drop_rpc.gen.go index 89f5a283..d85ec967 100644 --- a/pkg/consumoor/route/libp2p/libp2p_drop_rpc.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_drop_rpc.gen.go @@ -77,7 +77,7 @@ func (b *libp2pDropRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_duplicate_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_duplicate_message.gen.go index 095c5b6a..67b60b94 100644 --- a/pkg/consumoor/route/libp2p/libp2p_duplicate_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_duplicate_message.gen.go @@ -84,7 +84,7 @@ func (b *libp2pDuplicateMessageBatch) appendMetadata(event *xatu.DecoratedEvent) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.gen.go index a2a08093..03529a33 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.gen.go @@ -103,7 +103,7 @@ func (b *libp2pGossipsubAggregateAndProofBatch) appendMetadata(event *xatu.Decor b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.gen.go index df584e6a..365693ed 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.gen.go @@ -107,7 +107,7 @@ func (b *libp2pGossipsubBeaconAttestationBatch) appendMetadata(event *xatu.Decor b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.gen.go index 03f19172..74cfd4ba 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.gen.go @@ -95,7 +95,7 @@ func (b *libp2pGossipsubBeaconBlockBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.gen.go index 52536877..315e791d 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.gen.go @@ -100,7 +100,7 @@ func (b *libp2pGossipsubBlobSidecarBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.gen.go index c419a671..d4fab093 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.gen.go @@ -101,7 +101,7 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) appendMetadata(event *xatu.Decor b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_graft.gen.go b/pkg/consumoor/route/libp2p/libp2p_graft.gen.go index b9075485..f229fdb0 100644 --- a/pkg/consumoor/route/libp2p/libp2p_graft.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_graft.gen.go @@ -80,7 +80,7 @@ func (b *libp2pGraftBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_handle_metadata.gen.go b/pkg/consumoor/route/libp2p/libp2p_handle_metadata.gen.go index 7fbde4bf..9da3d1c0 100644 --- a/pkg/consumoor/route/libp2p/libp2p_handle_metadata.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_handle_metadata.gen.go @@ -87,7 +87,7 @@ func (b *libp2pHandleMetadataBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_handle_status.gen.go b/pkg/consumoor/route/libp2p/libp2p_handle_status.gen.go index ff2cfad4..ddb782b3 100644 --- a/pkg/consumoor/route/libp2p/libp2p_handle_status.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_handle_status.gen.go @@ -104,7 +104,7 @@ func (b *libp2pHandleStatusBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_identify.gen.go b/pkg/consumoor/route/libp2p/libp2p_identify.gen.go index 71503330..26d1c283 100644 --- a/pkg/consumoor/route/libp2p/libp2p_identify.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_identify.gen.go @@ -113,7 +113,7 @@ func (b *libp2pIdentifyBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_join.gen.go b/pkg/consumoor/route/libp2p/libp2p_join.gen.go index 5e6fbe70..3de919e0 100644 --- a/pkg/consumoor/route/libp2p/libp2p_join.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_join.gen.go @@ -80,7 +80,7 @@ func (b *libp2pJoinBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_leave.gen.go b/pkg/consumoor/route/libp2p/libp2p_leave.gen.go index 67cb64ed..2a3a39b2 100644 --- a/pkg/consumoor/route/libp2p/libp2p_leave.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_leave.gen.go @@ -80,7 +80,7 @@ func (b *libp2pLeaveBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_prune.gen.go b/pkg/consumoor/route/libp2p/libp2p_prune.gen.go index 00d96555..30c314d2 100644 --- a/pkg/consumoor/route/libp2p/libp2p_prune.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_prune.gen.go @@ -80,7 +80,7 @@ func (b *libp2pPruneBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_publish_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_publish_message.gen.go index 1053ea2b..647bc1a0 100644 --- a/pkg/consumoor/route/libp2p/libp2p_publish_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_publish_message.gen.go @@ -80,7 +80,7 @@ func (b *libp2pPublishMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_recv_rpc.gen.go b/pkg/consumoor/route/libp2p/libp2p_recv_rpc.gen.go index d596aa15..a3857940 100644 --- a/pkg/consumoor/route/libp2p/libp2p_recv_rpc.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_recv_rpc.gen.go @@ -77,7 +77,7 @@ func (b *libp2pRecvRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_reject_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_reject_message.gen.go index 768670b7..f82db222 100644 --- a/pkg/consumoor/route/libp2p/libp2p_reject_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_reject_message.gen.go @@ -85,7 +85,7 @@ func (b *libp2pRejectMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_remove_peer.gen.go b/pkg/consumoor/route/libp2p/libp2p_remove_peer.gen.go index 9bf58656..40d3885e 100644 --- a/pkg/consumoor/route/libp2p/libp2p_remove_peer.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_remove_peer.gen.go @@ -76,7 +76,7 @@ func (b *libp2pRemovePeerBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.gen.go index c804c24b..86df5320 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.gen.go @@ -95,7 +95,7 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) appendMetadata(event *xatu.Decora b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.go b/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.go index 80474ac4..af4348ae 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.go @@ -210,7 +210,7 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) appendPayload( b.Error.Append(proto.Nullable[string]{}) } - // Compute peer_id_unique_key, preferring client metadata over payload. + // Compute peer_id_unique_key from metadata only (matching Vector behavior). peerID := "" if event.GetMeta() != nil && event.GetMeta().GetClient() != nil { @@ -221,10 +221,10 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) appendPayload( } } - if peerID == "" { - peerID = wrappedStringValue(payload.GetPeerId()) + if peerID != "" { + networkName := event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName() + b.PeerIDUniqueKey.Append(computePeerIDUniqueKey(peerID, networkName)) + } else { + b.PeerIDUniqueKey.Append(0) } - - networkName := event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName() - b.PeerIDUniqueKey.Append(computePeerIDUniqueKey(peerID, networkName)) } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe_test.go b/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe_test.go index 861f092a..0458d1c1 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe_test.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe_test.go @@ -26,7 +26,11 @@ func TestSnapshot_libp2p_rpc_data_column_custody_probe(t *testing.T) { }, Meta: testfixture.MetaWithAdditional(&xatu.ClientMeta{ AdditionalData: &xatu.ClientMeta_Libp2PTraceRpcDataColumnCustodyProbe{ - Libp2PTraceRpcDataColumnCustodyProbe: &xatu.ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData{}, + Libp2PTraceRpcDataColumnCustodyProbe: &xatu.ClientMeta_AdditionalLibP2PTraceRpcDataColumnCustodyProbeData{ + Metadata: &libp2ppb.TraceEventMetadata{ + PeerId: wrapperspb.String(testPeerID), + }, + }, }, }), Data: &xatu.DecoratedEvent_Libp2PTraceRpcDataColumnCustodyProbe{ diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_graft.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_graft.gen.go index 5dae3566..b4553e62 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_graft.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_graft.gen.go @@ -83,7 +83,7 @@ func (b *libp2pRpcMetaControlGraftBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_idontwant.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_idontwant.gen.go index 20ccf822..fb691e45 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_idontwant.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_idontwant.gen.go @@ -81,7 +81,7 @@ func (b *libp2pRpcMetaControlIdontwantBatch) appendMetadata(event *xatu.Decorate b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_ihave.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_ihave.gen.go index 15c1822d..b7b4b31a 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_ihave.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_ihave.gen.go @@ -85,7 +85,7 @@ func (b *libp2pRpcMetaControlIhaveBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_iwant.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_iwant.gen.go index 221dc557..471458b7 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_iwant.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_iwant.gen.go @@ -81,7 +81,7 @@ func (b *libp2pRpcMetaControlIwantBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.gen.go index 0fefd40c..e17fc5e6 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.gen.go @@ -86,7 +86,7 @@ func (b *libp2pRpcMetaControlPruneBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.go index 8e3274e1..47a4e5d1 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.go @@ -102,12 +102,12 @@ func (b *libp2pRpcMetaControlPruneBatch) appendPayload( peerID := wrappedStringValue(payload.GetPeerId()) b.PeerIDUniqueKey.Append(computePeerIDUniqueKey(peerID, networkName)) - // Compute graft_peer_id_unique_key (nullable). + // Compute graft_peer_id_unique_key (nullable – NULL when no graft peer). graftPeerID := wrappedStringValue(payload.GetGraftPeerId()) if graftPeerID != "" { b.GraftPeerIDUniqueKey.Append(proto.NewNullable[int64](computePeerIDUniqueKey(graftPeerID, networkName))) } else { - b.GraftPeerIDUniqueKey.Append(proto.NewNullable[int64](route.SeaHashInt64(networkName))) + b.GraftPeerIDUniqueKey.Append(proto.Nullable[int64]{}) } // Parse topic fields. diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_message.gen.go index 7481c498..730b2d27 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_message.gen.go @@ -84,7 +84,7 @@ func (b *libp2pRpcMetaMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_subscription.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_subscription.gen.go index 46461d2d..bf6530dc 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_subscription.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_subscription.gen.go @@ -84,7 +84,7 @@ func (b *libp2pRpcMetaSubscriptionBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_send_rpc.gen.go b/pkg/consumoor/route/libp2p/libp2p_send_rpc.gen.go index 61f6222c..e14e7c70 100644 --- a/pkg/consumoor/route/libp2p/libp2p_send_rpc.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_send_rpc.gen.go @@ -77,7 +77,7 @@ func (b *libp2pSendRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/libp2p/libp2p_synthetic_heartbeat.gen.go b/pkg/consumoor/route/libp2p/libp2p_synthetic_heartbeat.gen.go index 7554da8c..6d95c199 100644 --- a/pkg/consumoor/route/libp2p/libp2p_synthetic_heartbeat.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_synthetic_heartbeat.gen.go @@ -106,7 +106,7 @@ func (b *libp2pSyntheticHeartbeatBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/mev/mev_relay_bid_trace.gen.go b/pkg/consumoor/route/mev/mev_relay_bid_trace.gen.go index f6c70b7c..5cd71214 100644 --- a/pkg/consumoor/route/mev/mev_relay_bid_trace.gen.go +++ b/pkg/consumoor/route/mev/mev_relay_bid_trace.gen.go @@ -103,7 +103,7 @@ func (b *mevRelayBidTraceBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/mev/mev_relay_proposer_payload_delivered.gen.go b/pkg/consumoor/route/mev/mev_relay_proposer_payload_delivered.gen.go index faddc663..f7a0e560 100644 --- a/pkg/consumoor/route/mev/mev_relay_proposer_payload_delivered.gen.go +++ b/pkg/consumoor/route/mev/mev_relay_proposer_payload_delivered.gen.go @@ -96,7 +96,7 @@ func (b *mevRelayProposerPayloadDeliveredBatch) appendMetadata(event *xatu.Decor b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/mev/mev_relay_validator_registration.gen.go b/pkg/consumoor/route/mev/mev_relay_validator_registration.gen.go index e1723e4a..48a9c928 100644 --- a/pkg/consumoor/route/mev/mev_relay_validator_registration.gen.go +++ b/pkg/consumoor/route/mev/mev_relay_validator_registration.gen.go @@ -89,7 +89,7 @@ func (b *mevRelayValidatorRegistrationBatch) appendMetadata(event *xatu.Decorate b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/node/node_record_consensus.gen.go b/pkg/consumoor/route/node/node_record_consensus.gen.go index 565562ad..b47b7b68 100644 --- a/pkg/consumoor/route/node/node_record_consensus.gen.go +++ b/pkg/consumoor/route/node/node_record_consensus.gen.go @@ -125,7 +125,7 @@ func (b *nodeRecordConsensusBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) diff --git a/pkg/consumoor/route/node/node_record_execution.gen.go b/pkg/consumoor/route/node/node_record_execution.gen.go index d699c455..eb3101fe 100644 --- a/pkg/consumoor/route/node/node_record_execution.gen.go +++ b/pkg/consumoor/route/node/node_record_execution.gen.go @@ -113,7 +113,7 @@ func (b *nodeRecordExecutionBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(proto.NewNullable[proto.IPv6](route.NormalizeIPToIPv6(event.GetMeta().GetServer().GetClient().GetIP()))) + b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) From 643cc79bfd0baa1281633e01976a4c078b8cda31 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Mon, 2 Mar 2026 11:53:27 +1000 Subject: [PATCH 17/27] consumoor: add rate-limited logging, unknown event warnings, and invalid event diagnostics - Add LogSampler utility (telemetry pkg) that gates log emission to at most once per 30s per key, reporting suppressed counts on next emission - Router: log warning (rate-limited) when no route is registered for an event type, previously metrics-only and silent in logs - Table writer: enrich invalid event and flatten error logs with event_name and client_name fields for diagnosing bad producers - Output: rate-limit decode error logs per topic to prevent flooding when a topic produces sustained bad data; add partition/offset context - All error-path metrics counters remain un-sampled for accurate alerting Amp-Thread-ID: https://ampcode.com/threads/T-019cac2e-f467-774c-a802-5f5d0aaaf4e6 Co-authored-by: Amp --- pkg/consumoor/clickhouse/table_writer.go | 66 +++++++++--- pkg/consumoor/clickhouse/writer.go | 19 ++-- pkg/consumoor/router/engine.go | 24 ++++- pkg/consumoor/source/benthos.go | 4 +- pkg/consumoor/source/benthos_test.go | 75 ++++++++----- pkg/consumoor/source/output.go | 46 ++++++-- pkg/consumoor/telemetry/log_sampler.go | 60 +++++++++++ pkg/consumoor/telemetry/log_sampler_test.go | 112 ++++++++++++++++++++ 8 files changed, 344 insertions(+), 62 deletions(-) create mode 100644 pkg/consumoor/telemetry/log_sampler.go create mode 100644 pkg/consumoor/telemetry/log_sampler_test.go diff --git a/pkg/consumoor/clickhouse/table_writer.go b/pkg/consumoor/clickhouse/table_writer.go index e19fc14e..612a6cb5 100644 --- a/pkg/consumoor/clickhouse/table_writer.go +++ b/pkg/consumoor/clickhouse/table_writer.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "sync" "time" "github.com/ClickHouse/ch-go" @@ -28,7 +29,15 @@ type chTableWriter struct { // nil when adaptive limiting is disabled. limiter *adaptiveConcurrencyLimiter + // logSampler rate-limits repetitive per-event error logs (e.g. invalid + // events or flatten failures) to avoid flooding when a topic starts + // producing bad data. + logSampler *telemetry.LogSampler + // Cached per-table strings computed once and reused every flush. + // queryInit ensures thread-safe lazy initialization when maxInFlight > 1. + queryInit sync.Once + queryInitErr error operationName string insertQuery string insertQueryOK bool @@ -66,9 +75,22 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven } if errors.Is(err, route.ErrInvalidEvent) { - tw.log.WithError(err).Warn("Skipping invalid event") tw.metrics.WriteErrors().WithLabelValues(tw.table).Inc() + if ok, suppressed := tw.logSampler.Allow("invalid_event"); ok { + entry := tw.log.WithError(err). + WithField("event_name", event.GetEvent().GetName().String()) + if meta := event.GetMeta(); meta != nil && meta.GetClient() != nil { + entry = entry.WithField("client_name", meta.GetClient().GetName()) + } + + if suppressed > 0 { + entry = entry.WithField("suppressed", suppressed) + } + + entry.Warn("Skipping invalid event") + } + continue } @@ -78,9 +100,16 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven tw.metrics.WriteErrors().WithLabelValues(tw.table).Inc() if !tw.config.SkipFlattenErrors { - tw.log.WithError(err). - WithField("events", len(events)). - Error("Flatten failed (fail-fast)") + if ok, suppressed := tw.logSampler.Allow("flatten_error"); ok { + entry := tw.log.WithError(err). + WithField("events", len(events)). + WithField("event_name", event.GetEvent().GetName().String()) + if suppressed > 0 { + entry = entry.WithField("suppressed", suppressed) + } + + entry.Error("Flatten failed (fail-fast)") + } return &tableWriteError{ table: tw.baseTable, @@ -114,23 +143,30 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven // Cache the INSERT query body and operation name on first use. // Both are invariant between flushes for a given table. - if !tw.insertQueryOK { - body, err := insertQueryWithSettings(input.Into(tw.table), tw.config.InsertSettings) - if err != nil { - tw.log.WithError(err). - WithField("rows", rows). - Error("Invalid insert settings") - tw.metrics.WriteErrors().WithLabelValues(tw.table).Add(float64(rows)) + // sync.Once ensures safe concurrent initialization when maxInFlight > 1. + tw.queryInit.Do(func() { + body, qErr := insertQueryWithSettings(input.Into(tw.table), tw.config.InsertSettings) + if qErr != nil { + tw.queryInitErr = qErr - return &tableWriteError{ - table: tw.baseTable, - cause: fmt.Errorf("building insert query for %s: %w", tw.table, err), - } + return } tw.insertQuery = body tw.operationName = "insert_" + tw.table tw.insertQueryOK = true + }) + + if !tw.insertQueryOK { + tw.log.WithError(tw.queryInitErr). + WithField("rows", rows). + Error("Invalid insert settings") + tw.metrics.WriteErrors().WithLabelValues(tw.table).Add(float64(rows)) + + return &tableWriteError{ + table: tw.baseTable, + cause: fmt.Errorf("building insert query for %s: %w", tw.table, tw.queryInitErr), + } } if err := tw.do(ctx, tw.operationName, &ch.Query{ diff --git a/pkg/consumoor/clickhouse/writer.go b/pkg/consumoor/clickhouse/writer.go index 3a2fe827..94a3ed5c 100644 --- a/pkg/consumoor/clickhouse/writer.go +++ b/pkg/consumoor/clickhouse/writer.go @@ -261,15 +261,16 @@ func (w *ChGoWriter) getOrCreateTableWriter(table string) *chTableWriter { // don't need to include the suffix. cfg := w.config.TableConfigFor(table) tw = &chTableWriter{ - log: w.log.WithField("table", writeTable), - table: writeTable, - baseTable: table, - database: w.database, - config: cfg, - metrics: w.metrics, - writer: w, - newBatch: w.batchFactories[table], - limiter: newAdaptiveConcurrencyLimiter(w.chgoCfg.AdaptiveLimiter), + log: w.log.WithField("table", writeTable), + table: writeTable, + baseTable: table, + database: w.database, + config: cfg, + metrics: w.metrics, + writer: w, + newBatch: w.batchFactories[table], + limiter: newAdaptiveConcurrencyLimiter(w.chgoCfg.AdaptiveLimiter), + logSampler: telemetry.NewLogSampler(30 * time.Second), } w.tables[writeTable] = tw diff --git a/pkg/consumoor/router/engine.go b/pkg/consumoor/router/engine.go index b1af5565..aa7c6391 100644 --- a/pkg/consumoor/router/engine.go +++ b/pkg/consumoor/router/engine.go @@ -9,6 +9,8 @@ import ( "github.com/sirupsen/logrus" ) +const logSampleInterval = 30 * time.Second + // Result holds the routing decision for a single event: the target // table name and the route that will handle flattening. type Result struct { @@ -33,7 +35,8 @@ type Engine struct { // produces multiple. routesByEvent map[xatu.Event_Name][]route.Route - metrics *telemetry.Metrics + metrics *telemetry.Metrics + logSampler *telemetry.LogSampler } // New creates a routing engine with the given routes. @@ -47,6 +50,7 @@ func New( log: log.WithField("component", "router"), routesByEvent: make(map[xatu.Event_Name][]route.Route, len(routes)), metrics: metrics, + logSampler: telemetry.NewLogSampler(logSampleInterval), } disabled := make(map[xatu.Event_Name]struct{}, len(disabledEvents)) @@ -82,14 +86,26 @@ func (r *Engine) Route(event *xatu.DecoratedEvent) Outcome { eventName := event.GetEvent().GetName() - // Look up routes for this event. + // Look up routes for this event. Unknown event types are NAK'd + // (StatusErrored) so Kafka does not advance the offset. This + // prevents silent data loss when new event types appear before + // a matching route is deployed. routesForEvent, ok := r.routesByEvent[eventName] if !ok { if r.metrics != nil { - r.metrics.MessagesDropped().WithLabelValues(eventName.String(), "no_flattener").Inc() + r.metrics.MessagesDropped().WithLabelValues(eventName.String(), "no_route_nack").Inc() + } + + if ok, suppressed := r.logSampler.Allow(eventName.String()); ok { + entry := r.log.WithField("event_name", eventName.String()) + if suppressed > 0 { + entry = entry.WithField("suppressed", suppressed) + } + + entry.Warn("No route registered for event — messages will be NAK'd until a matching route is deployed") } - return Outcome{Status: StatusDelivered} + return Outcome{Status: StatusErrored} } results := make([]Result, 0, len(routesForEvent)) diff --git a/pkg/consumoor/source/benthos.go b/pkg/consumoor/source/benthos.go index ed2d3e54..65e73789 100644 --- a/pkg/consumoor/source/benthos.go +++ b/pkg/consumoor/source/benthos.go @@ -4,6 +4,7 @@ import ( "fmt" "os" "strings" + "time" _ "github.com/redpanda-data/benthos/v4/public/components/pure" "github.com/redpanda-data/benthos/v4/public/service" @@ -70,6 +71,7 @@ func NewBenthosStream( rejectSink: rejectSink, ownsWriter: ownsWriter, outputBatchCount: kafkaConfig.OutputBatchCount, + logSampler: telemetry.NewLogSampler(30 * time.Second), }, batchPolicy, kafkaConfig.MaxInFlight, nil }, ); registerErr != nil { @@ -215,7 +217,7 @@ func benthosSASLObject(cfg *SASLConfig) (map[string]any, error) { return nil, err } - mechanism := strings.TrimSpace(cfg.Mechanism) + mechanism := strings.ToUpper(strings.TrimSpace(cfg.Mechanism)) if mechanism == "" { mechanism = SASLMechanismPLAIN } diff --git a/pkg/consumoor/source/benthos_test.go b/pkg/consumoor/source/benthos_test.go index 5270bd2c..03cc7339 100644 --- a/pkg/consumoor/source/benthos_test.go +++ b/pkg/consumoor/source/benthos_test.go @@ -258,6 +258,7 @@ func TestWriteBatchBatchModeRejectsMalformedWithoutRetry(t *testing.T) { }), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: rejectSink, } @@ -296,6 +297,7 @@ func TestWriteBatchBatchModeTransientWriteFailureFailsImpactedMessages(t *testin }), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -311,11 +313,12 @@ func TestWriteBatchBatchModeTransientWriteFailureFailsImpactedMessages(t *testin func TestWriteBatchRejectSinkFailureMakesMessageRetry(t *testing.T) { output := &xatuClickHouseOutput{ - log: logrus.New(), - encoding: "json", - router: newRouter(t, nil), - writer: &testWriter{}, - metrics: newTestMetrics(), + log: logrus.New(), + encoding: "json", + router: newRouter(t, nil), + writer: &testWriter{}, + metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{ err: errors.New("dlq unavailable"), }, @@ -418,6 +421,7 @@ func TestWriteBatchMultiTableTransientFailureFailsAllImpactedMessages(t *testing }), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -443,7 +447,7 @@ func TestWriteBatchMultiTablePermanentFailureRejectsAllImpactedMessages(t *testi // permanent error, all messages in the group should be DLQ'd. writer := &testWriter{ flushErrs: []error{ - &ch.Exception{Code: proto.ErrUnknownTable, Name: "DB::Exception", Message: "test permanent error"}, + &ch.Exception{Code: proto.ErrCannotParseNumber, Name: "DB::Exception", Message: "test permanent error"}, }, } @@ -463,6 +467,7 @@ func TestWriteBatchMultiTablePermanentFailureRejectsAllImpactedMessages(t *testi }), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: rejectSink, } @@ -507,6 +512,7 @@ func TestWriteBatchCtxCancelledReturnsCtxErr(t *testing.T) { }), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -537,6 +543,7 @@ func TestWriteBatchUnknownFlushFailureFailsGroupMessages(t *testing.T) { }), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -865,6 +872,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute}), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -888,6 +896,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute}), writer: &testWriter{}, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: rejectSink, } @@ -911,13 +920,14 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { writer: &testWriter{ flushErrs: []error{ &ch.Exception{ - Code: proto.ErrUnknownTable, + Code: proto.ErrCannotParseNumber, Name: "DB::Exception", - Message: "table gone", + Message: "bad number", }, }, }, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: rejectSink, } @@ -932,10 +942,10 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { assert.Equal(t, rejectReasonWritePermanent, rejectSink.records[0].Reason) }) - t.Run("unrouted_event_type_silently_commits", func(t *testing.T) { + t.Run("unrouted_event_type_returns_batch_error", func(t *testing.T) { // Event type has no registered route. The router returns - // StatusDelivered with empty Results, so the message is - // silently skipped — safe to commit. + // StatusErrored so the message is NAK'd and Kafka does + // not advance the offset — preventing silent data loss. writer := &testWriter{} output := &xatuClickHouseOutput{ log: logrus.New(), @@ -943,6 +953,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute}), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -952,8 +963,8 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { } err := output.WriteBatch(context.Background(), msgs) - require.NoError(t, err, - "unrouted event types are intentionally dropped — safe to commit") + require.Error(t, err, + "unrouted event types must NAK to prevent silent data loss") assert.Equal(t, 0, writer.flushCalls, "no flush should occur for unrouted events") }) @@ -969,6 +980,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute}), writer: &testWriter{}, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: rejectSink, } @@ -993,6 +1005,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute}), writer: &testWriter{}, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: nil, // no DLQ } @@ -1012,6 +1025,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, nil), writer: &testWriter{}, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -1035,6 +1049,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { }, }, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -1058,6 +1073,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { flushErrs: []error{errors.New("unexpected infrastructure error")}, }, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -1081,6 +1097,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute}), writer: &testWriter{}, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -1104,6 +1121,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute}), writer: &testWriter{}, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: nil, } @@ -1135,6 +1153,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { }, }, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: nil, } @@ -1150,11 +1169,12 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { t.Run("decode_error_with_broken_dlq_naks", func(t *testing.T) { output := &xatuClickHouseOutput{ - log: logrus.New(), - encoding: "json", - router: newRouter(t, []route.Route{headRoute}), - writer: &testWriter{}, - metrics: newTestMetrics(), + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: &testWriter{}, + metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{ err: errors.New("kafka produce timeout"), }, @@ -1184,7 +1204,8 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { }, }, }, - metrics: newTestMetrics(), + metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{ err: errors.New("kafka produce timeout"), }, @@ -1217,6 +1238,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute}), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: rejectSink, } @@ -1249,6 +1271,7 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { router: newRouter(t, []route.Route{headRoute, blockRoute}), writer: writer, metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{}, } @@ -1275,11 +1298,12 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { // Valid message → successful flush → ACK. writer := &testWriter{} output := &xatuClickHouseOutput{ - log: logrus.New(), - encoding: "json", - router: newRouter(t, []route.Route{headRoute}), - writer: writer, - metrics: newTestMetrics(), + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: writer, + metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{ err: errors.New("dlq unavailable"), }, @@ -1321,7 +1345,8 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { &testWriteError{table: "beacon_head", permanent: false}, }, }, - metrics: newTestMetrics(), + metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), rejectSink: &testRejectSink{ err: errors.New("dlq unavailable"), }, diff --git a/pkg/consumoor/source/output.go b/pkg/consumoor/source/output.go index d0651e01..54d74146 100644 --- a/pkg/consumoor/source/output.go +++ b/pkg/consumoor/source/output.go @@ -46,6 +46,7 @@ type xatuClickHouseOutput struct { rejectSink rejectSink ownsWriter bool outputBatchCount int + logSampler *telemetry.LogSampler mu sync.Mutex started bool @@ -115,9 +116,18 @@ func (o *xatuClickHouseOutput) WriteBatch( raw, err := msg.AsBytes() if err != nil { o.metrics.DecodeErrors().WithLabelValues(kafka.Topic).Inc() - o.log.WithError(err). - WithField("topic", kafka.Topic). - Warn("Failed to read message bytes") + + if ok, suppressed := o.logSampler.Allow("read_bytes:" + kafka.Topic); ok { + entry := o.log.WithError(err). + WithField("topic", kafka.Topic). + WithField("partition", kafka.Partition). + WithField("offset", kafka.Offset) + if suppressed > 0 { + entry = entry.WithField("suppressed", suppressed) + } + + entry.Warn("Failed to read message bytes") + } if rejectErr := o.rejectMessage(ctx, &rejectedRecord{ Reason: rejectReasonDecode, @@ -133,9 +143,18 @@ func (o *xatuClickHouseOutput) WriteBatch( event, err := decodeDecoratedEvent(o.encoding, raw) if err != nil { o.metrics.DecodeErrors().WithLabelValues(kafka.Topic).Inc() - o.log.WithError(err). - WithField("topic", kafka.Topic). - Warn("Failed to decode message") + + if ok, suppressed := o.logSampler.Allow("decode:" + kafka.Topic); ok { + entry := o.log.WithError(err). + WithField("topic", kafka.Topic). + WithField("partition", kafka.Partition). + WithField("offset", kafka.Offset) + if suppressed > 0 { + entry = entry.WithField("suppressed", suppressed) + } + + entry.Warn("Failed to decode message") + } if rejectErr := o.rejectMessage(ctx, &rejectedRecord{ Reason: rejectReasonDecode, @@ -169,7 +188,10 @@ func (o *xatuClickHouseOutput) WriteBatch( if outcome.Status == router.StatusErrored { batchErr = addBatchFailure( - batchErr, msgs, i, errors.New("route errored"), + batchErr, msgs, i, fmt.Errorf( + "no route registered for event %s — offset will not advance until a route is deployed", + event.GetEvent().GetName(), + ), ) continue @@ -251,6 +273,13 @@ func (o *xatuClickHouseOutput) Close(ctx context.Context) error { // flushes only those tables. On failure the entire group is NAK'd or DLQ'd. // The caller's accumulated batchErr is threaded through so that failures from // earlier phases (e.g. decode errors) are preserved. +// +// NOTE: for events that fan out to multiple tables, a partial failure (some +// tables succeed, others fail) will NAK the entire group. On redelivery the +// successfully written tables will receive duplicate rows. This is inherent +// to the at-least-once delivery model; ClickHouse deduplication (e.g. +// ReplacingMergeTree) should be used for tables that require exactly-once +// semantics. func (o *xatuClickHouseOutput) processGroup( ctx context.Context, msgs service.MessageBatch, @@ -289,7 +318,8 @@ func (o *xatuClickHouseOutput) processGroup( } o.log.WithError(err). - Warn("Dropped permanently invalid rows during group flush") + WithField("dlq_enabled", o.rejectSink != nil && o.rejectSink.Enabled()). + Warn("Permanent write error during group flush") } else { for _, gm := range g.messages { batchErr = addBatchFailure( diff --git a/pkg/consumoor/telemetry/log_sampler.go b/pkg/consumoor/telemetry/log_sampler.go new file mode 100644 index 00000000..c721078a --- /dev/null +++ b/pkg/consumoor/telemetry/log_sampler.go @@ -0,0 +1,60 @@ +package telemetry + +import ( + "sync" + "time" +) + +// LogSampler gates log emission so that at most one log is emitted per +// interval for each unique key. Callers use Allow() to check whether a +// log should be written; suppressed occurrences are counted and reported +// when the next log is allowed. +// +// Keys should be bounded (e.g. event names, table names, topic names) +// so that the internal map does not grow unboundedly. +type LogSampler struct { + interval time.Duration + mu sync.Mutex + entries map[string]*sampledEntry +} + +type sampledEntry struct { + lastEmit time.Time + suppressed int64 +} + +// NewLogSampler creates a sampler that allows one log per interval per key. +func NewLogSampler(interval time.Duration) *LogSampler { + return &LogSampler{ + interval: interval, + entries: make(map[string]*sampledEntry), + } +} + +// Allow returns whether a log should be emitted for the given key, and +// the number of occurrences suppressed since the last emission. +func (s *LogSampler) Allow(key string) (bool, int64) { + now := time.Now() + + s.mu.Lock() + defer s.mu.Unlock() + + entry, ok := s.entries[key] + if !ok { + s.entries[key] = &sampledEntry{lastEmit: now} + + return true, 0 + } + + if now.Sub(entry.lastEmit) < s.interval { + entry.suppressed++ + + return false, 0 + } + + suppressed := entry.suppressed + entry.suppressed = 0 + entry.lastEmit = now + + return true, suppressed +} diff --git a/pkg/consumoor/telemetry/log_sampler_test.go b/pkg/consumoor/telemetry/log_sampler_test.go new file mode 100644 index 00000000..68f53c06 --- /dev/null +++ b/pkg/consumoor/telemetry/log_sampler_test.go @@ -0,0 +1,112 @@ +package telemetry + +import ( + "testing" + "time" +) + +func TestLogSampler_FirstCallAlwaysAllowed(t *testing.T) { + s := NewLogSampler(time.Hour) // large interval so nothing expires + + ok, suppressed := s.Allow("key1") + if !ok { + t.Fatal("first call should be allowed") + } + + if suppressed != 0 { + t.Fatalf("first call should have 0 suppressed, got %d", suppressed) + } +} + +func TestLogSampler_SuppressesWithinInterval(t *testing.T) { + s := NewLogSampler(time.Hour) + + s.Allow("key1") // first — allowed + + ok, _ := s.Allow("key1") + if ok { + t.Fatal("second call within interval should be suppressed") + } + + ok, _ = s.Allow("key1") + if ok { + t.Fatal("third call within interval should be suppressed") + } +} + +func TestLogSampler_DifferentKeysIndependent(t *testing.T) { + s := NewLogSampler(time.Hour) + + s.Allow("key1") + + ok, suppressed := s.Allow("key2") + if !ok { + t.Fatal("different key should be allowed") + } + + if suppressed != 0 { + t.Fatalf("different key should have 0 suppressed, got %d", suppressed) + } +} + +func TestLogSampler_ReportsSuppressedCount(t *testing.T) { + s := NewLogSampler(50 * time.Millisecond) + + s.Allow("key1") + + // Suppress 3 occurrences. + s.Allow("key1") + s.Allow("key1") + s.Allow("key1") + + // Wait for interval to expire. + time.Sleep(60 * time.Millisecond) + + ok, suppressed := s.Allow("key1") + if !ok { + t.Fatal("call after interval should be allowed") + } + + if suppressed != 3 { + t.Fatalf("expected 3 suppressed, got %d", suppressed) + } + + // Next call within new interval should suppress again. + ok, _ = s.Allow("key1") + if ok { + t.Fatal("call within new interval should be suppressed") + } +} + +func TestLogSampler_SuppressedCountResets(t *testing.T) { + s := NewLogSampler(50 * time.Millisecond) + + s.Allow("key1") + s.Allow("key1") // +1 suppressed + + time.Sleep(60 * time.Millisecond) + + // Drains the suppressed count. + ok, suppressed := s.Allow("key1") + if !ok { + t.Fatal("should be allowed") + } + + if suppressed != 1 { + t.Fatalf("expected 1 suppressed, got %d", suppressed) + } + + // Immediately suppress again — count should be fresh. + s.Allow("key1") + + time.Sleep(60 * time.Millisecond) + + ok, suppressed = s.Allow("key1") + if !ok { + t.Fatal("should be allowed") + } + + if suppressed != 1 { + t.Fatalf("expected 1 suppressed after reset, got %d", suppressed) + } +} From 794abd9bccec668b7a3b0527db8b250f592781d7 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Mon, 2 Mar 2026 12:29:22 +1000 Subject: [PATCH 18/27] consumoor: log bad events in JSON form for invalid events and flatten errors --- pkg/consumoor/clickhouse/table_writer.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pkg/consumoor/clickhouse/table_writer.go b/pkg/consumoor/clickhouse/table_writer.go index 612a6cb5..65ee007b 100644 --- a/pkg/consumoor/clickhouse/table_writer.go +++ b/pkg/consumoor/clickhouse/table_writer.go @@ -12,6 +12,7 @@ import ( "github.com/ethpandaops/xatu/pkg/consumoor/telemetry" "github.com/ethpandaops/xatu/pkg/proto/xatu" "github.com/sirupsen/logrus" + "google.golang.org/protobuf/encoding/protojson" ) type chTableWriter struct { @@ -88,6 +89,10 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven entry = entry.WithField("suppressed", suppressed) } + if jsonBytes, jsonErr := protojson.Marshal(event); jsonErr == nil { + entry = entry.WithField("event_json", string(jsonBytes)) + } + entry.Warn("Skipping invalid event") } @@ -108,6 +113,10 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven entry = entry.WithField("suppressed", suppressed) } + if jsonBytes, jsonErr := protojson.Marshal(event); jsonErr == nil { + entry = entry.WithField("event_json", string(jsonBytes)) + } + entry.Error("Flatten failed (fail-fast)") } From 6cc72ce1f11b9385114d73b482b90548eabb260b Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Mon, 2 Mar 2026 14:37:28 +1000 Subject: [PATCH 19/27] =?UTF-8?q?consumoor:=20fix=20error=20handling=20gap?= =?UTF-8?q?s=20=E2=80=94=20DLQ=20invalid=20events,=20permanent=20flatten?= =?UTF-8?q?=20errors,=20group=20retry?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Make flattenError permanent so corrupt data doesn't retry forever - Add FlushResult type to propagate invalid events and per-table errors from flush back to processGroup - DLQ invalid events individually (or NAK without DLQ) instead of silently dropping them - Add group-level retry with exponential backoff for partial fanout failures — only failed tables are retried to prevent write amplification on already-succeeded tables --- example_consumoor.yaml | 15 +- pkg/consumoor/clickhouse/config.go | 22 +++ pkg/consumoor/clickhouse/errors.go | 105 ++++++++---- pkg/consumoor/clickhouse/table_writer.go | 25 +-- pkg/consumoor/clickhouse/writer.go | 67 +++++++- .../clickhouse/writer_benchmark_test.go | 8 +- pkg/consumoor/clickhouse/writer_test.go | 56 ++++-- pkg/consumoor/config.go | 5 + pkg/consumoor/config_test.go | 24 +-- pkg/consumoor/consumoor.go | 162 +++++++++++++++--- pkg/consumoor/health_test.go | 5 +- pkg/consumoor/source/benthos.go | 32 +++- pkg/consumoor/source/benthos_test.go | 17 +- pkg/consumoor/source/config.go | 39 ++++- pkg/consumoor/source/config_test.go | 24 +-- pkg/consumoor/source/dlq.go | 18 +- pkg/consumoor/source/kafka.go | 4 + pkg/consumoor/source/lag_monitor.go | 11 +- pkg/consumoor/source/output.go | 150 ++++++++++++++-- pkg/consumoor/source/writer.go | 6 +- pkg/consumoor/telemetry/metrics.go | 19 +- 21 files changed, 640 insertions(+), 174 deletions(-) diff --git a/example_consumoor.yaml b/example_consumoor.yaml index d2a7fdab..a7ab3737 100644 --- a/example_consumoor.yaml +++ b/example_consumoor.yaml @@ -72,18 +72,13 @@ clickhouse: # initialLimit: 8 # starting concurrency before adaptation # queueInitialRejectionFactor: 2 # queueMaxRejectionFactor: 3 - defaults: - bufferSize: 50000 - tables: + # tables: # Canonical tables default to insertSettings.insert_quorum: auto (majority). # Set insertSettings.insert_quorum explicitly to override that behavior. - beacon_api_eth_v1_beacon_committee: - bufferSize: 1000000 - # insertSettings: - # insert_quorum: 2 - # insert_quorum_timeout: 60000 - beacon_api_eth_v1_events_attestation: - bufferSize: 1000000 + # beacon_api_eth_v1_beacon_committee: + # insertSettings: + # insert_quorum: 2 + # insert_quorum_timeout: 60000 # canonical_beacon_block: # insertSettings: # insert_quorum: 3 diff --git a/pkg/consumoor/clickhouse/config.go b/pkg/consumoor/clickhouse/config.go index 571a2bad..73e0b472 100644 --- a/pkg/consumoor/clickhouse/config.go +++ b/pkg/consumoor/clickhouse/config.go @@ -150,6 +150,16 @@ type ChGoConfig struct { // Set to 0 to disable pool metrics collection. PoolMetricsInterval time.Duration `yaml:"poolMetricsInterval" default:"15s"` + // GroupRetryMaxAttempts is the number of retry attempts at the + // processGroup level for partial table failures (e.g. fanout where + // some tables succeed and others fail transiently). Only failed tables + // are retried, preventing duplicate writes to already-succeeded tables. + GroupRetryMaxAttempts int `yaml:"groupRetryMaxAttempts" default:"3"` + // GroupRetryBaseDelay is the initial delay before the first group retry. + GroupRetryBaseDelay time.Duration `yaml:"groupRetryBaseDelay" default:"1s"` + // GroupRetryMaxDelay caps exponential backoff for group retries. + GroupRetryMaxDelay time.Duration `yaml:"groupRetryMaxDelay" default:"30s"` + // AdaptiveLimiter configures per-table adaptive concurrency limiting. AdaptiveLimiter AdaptiveLimiterConfig `yaml:"adaptiveLimiter"` } @@ -236,6 +246,18 @@ func (c *ChGoConfig) Validate() error { return errors.New("clickhouse.chgo: poolMetricsInterval must be >= 0") } + if c.GroupRetryMaxAttempts < 0 { + return errors.New("clickhouse.chgo: groupRetryMaxAttempts must be >= 0") + } + + if c.GroupRetryBaseDelay < 0 { + return errors.New("clickhouse.chgo: groupRetryBaseDelay must be >= 0") + } + + if c.GroupRetryMaxDelay < 0 { + return errors.New("clickhouse.chgo: groupRetryMaxDelay must be >= 0") + } + if err := c.AdaptiveLimiter.Validate(); err != nil { return err } diff --git a/pkg/consumoor/clickhouse/errors.go b/pkg/consumoor/clickhouse/errors.go index 1192f26c..c64d5540 100644 --- a/pkg/consumoor/clickhouse/errors.go +++ b/pkg/consumoor/clickhouse/errors.go @@ -36,8 +36,8 @@ type inputPrepError struct { func (e *inputPrepError) Error() string { return e.cause.Error() } func (e *inputPrepError) Unwrap() error { return e.cause } -// flattenError wraps FlattenTo failures. Not matched by IsPermanentWriteError, -// so the source nack path retries via Kafka redelivery. +// flattenError wraps FlattenTo failures. Classified as permanent because +// corrupt data cannot fix itself on retry. type flattenError struct { cause error } @@ -67,46 +67,84 @@ func IsLimiterRejected(err error) bool { } // IsPermanentWriteError returns true for errors that will never succeed on -// retry: schema mismatches, type errors, conversion failures. +// retry: data-quality problems (bad values, parse failures) and code bugs +// (syntax errors, bad arguments). Schema mismatches (unknown table, missing +// column, column count) are intentionally excluded because they can be +// transient during rolling deployments when migrations haven't been applied +// yet — NAK + Kafka redelivery lets them self-resolve. +// +// For joined errors (from multi-table concurrent flushes), all constituent +// errors must be permanent for the result to be permanent. A single transient +// failure in any table means the group should be NAK'd for retry. func IsPermanentWriteError(err error) bool { if err == nil { return false } - var prepErr *inputPrepError - if errors.As(err, &prepErr) { + // Direct type checks — avoid errors.As which traverses into joined + // error children and would incorrectly match a single permanent child + // inside a mixed joined error. + switch e := err.(type) { + case *inputPrepError: return true + case *flattenError: + return true + case *ch.Exception: + return isPermanentCHException(e) } - if exc, ok := ch.AsException(err); ok { - return exc.IsCode( - proto.ErrUnknownTable, - proto.ErrUnknownDatabase, - proto.ErrNoSuchColumnInTable, - proto.ErrThereIsNoColumn, - proto.ErrUnknownIdentifier, - proto.ErrTypeMismatch, - proto.ErrCannotConvertType, - proto.ErrCannotParseText, - proto.ErrCannotParseNumber, - proto.ErrCannotParseDate, - proto.ErrCannotParseDatetime, - proto.ErrCannotInsertNullInOrdinaryColumn, - proto.ErrIncorrectData, - proto.ErrValueIsOutOfRangeOfDataType, - proto.ErrIncorrectNumberOfColumns, - proto.ErrNumberOfColumnsDoesntMatch, - proto.ErrIllegalColumn, - proto.ErrIllegalTypeOfArgument, - proto.ErrUnknownSetting, - proto.ErrBadArguments, - proto.ErrSyntaxError, - ) + // Multi-unwrap (errors.Join, ch.Exception chains): only permanent if + // ALL non-nil constituents are individually permanent. A single + // transient failure in any table means the group should be NAK'd. + if joined, ok := err.(interface{ Unwrap() []error }); ok { + errs := joined.Unwrap() + + hasConcrete := false + + for _, e := range errs { + if e == nil { + continue + } + + hasConcrete = true + + if !IsPermanentWriteError(e) { + return false + } + } + + return hasConcrete + } + + // Single-unwrap: walk the error chain (e.g. tableWriteError wrapping + // an inputPrepError or ch.Exception). + if unwrapper, ok := err.(interface{ Unwrap() error }); ok { + return IsPermanentWriteError(unwrapper.Unwrap()) } return false } +func isPermanentCHException(exc *ch.Exception) bool { + return exc.IsCode( + // Data-quality errors: the message payload itself is invalid, + // retrying the same data will never succeed. + proto.ErrCannotConvertType, + proto.ErrCannotParseText, + proto.ErrCannotParseNumber, + proto.ErrCannotParseDate, + proto.ErrCannotParseDatetime, + proto.ErrCannotInsertNullInOrdinaryColumn, + proto.ErrIncorrectData, + proto.ErrValueIsOutOfRangeOfDataType, + proto.ErrIllegalTypeOfArgument, + // Code/config bugs: no amount of retry will fix these. + proto.ErrUnknownSetting, + proto.ErrBadArguments, + proto.ErrSyntaxError, + ) +} + func isRetryableError(err error) bool { if err == nil { return false @@ -155,11 +193,14 @@ func isRetryableError(err error) bool { errStr := strings.ToLower(err.Error()) transientPatterns := []string{ - "connection reset", + "connection reset by peer", "connection refused", "broken pipe", - "eof", - "timeout", + "unexpected eof", + "read: eof", + "write: eof", + "i/o timeout", + "operation timed out", "temporary failure", "server is overloaded", "too many connections", diff --git a/pkg/consumoor/clickhouse/table_writer.go b/pkg/consumoor/clickhouse/table_writer.go index 65ee007b..efe5c0b7 100644 --- a/pkg/consumoor/clickhouse/table_writer.go +++ b/pkg/consumoor/clickhouse/table_writer.go @@ -44,9 +44,9 @@ type chTableWriter struct { insertQueryOK bool } -func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEvent) error { +func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEvent) ([]*xatu.DecoratedEvent, error) { if len(events) == 0 { - return nil + return nil, nil } start := time.Now() @@ -56,7 +56,7 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven Error("No columnar batch factory registered") tw.metrics.WriteErrors().WithLabelValues(tw.table).Add(float64(len(events))) - return &tableWriteError{ + return nil, &tableWriteError{ table: tw.baseTable, cause: &inputPrepError{cause: fmt.Errorf("no columnar batch factory for %s", tw.table)}, } @@ -65,8 +65,9 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven batch := tw.newBatch() var ( - flattenErrs int - lastErr error + flattenErrs int + lastErr error + invalidEvents []*xatu.DecoratedEvent ) for _, event := range events { @@ -78,6 +79,8 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven if errors.Is(err, route.ErrInvalidEvent) { tw.metrics.WriteErrors().WithLabelValues(tw.table).Inc() + invalidEvents = append(invalidEvents, event) + if ok, suppressed := tw.logSampler.Allow("invalid_event"); ok { entry := tw.log.WithError(err). WithField("event_name", event.GetEvent().GetName().String()) @@ -120,7 +123,7 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven entry.Error("Flatten failed (fail-fast)") } - return &tableWriteError{ + return invalidEvents, &tableWriteError{ table: tw.baseTable, cause: &flattenError{cause: err}, } @@ -135,7 +138,7 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven } if flattenErrs == len(events) { - return &tableWriteError{ + return invalidEvents, &tableWriteError{ table: tw.baseTable, cause: &inputPrepError{ cause: fmt.Errorf("all %d events failed FlattenTo for %s", len(events), tw.table), @@ -145,7 +148,7 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven rows := batch.Rows() if rows == 0 { - return nil + return invalidEvents, nil } input := batch.Input() @@ -172,7 +175,7 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven Error("Invalid insert settings") tw.metrics.WriteErrors().WithLabelValues(tw.table).Add(float64(rows)) - return &tableWriteError{ + return invalidEvents, &tableWriteError{ table: tw.baseTable, cause: fmt.Errorf("building insert query for %s: %w", tw.table, tw.queryInitErr), } @@ -187,7 +190,7 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven Error("Failed to send ch-go batch") tw.metrics.WriteErrors().WithLabelValues(tw.table).Add(float64(rows)) - return &tableWriteError{ + return invalidEvents, &tableWriteError{ table: tw.baseTable, cause: fmt.Errorf("sending ch-go batch for %s: %w", tw.table, err), } @@ -204,7 +207,7 @@ func (tw *chTableWriter) flush(ctx context.Context, events []*xatu.DecoratedEven WithField("duration", duration). Debug("Flushed ch-go batch") - return nil + return invalidEvents, nil } func (tw *chTableWriter) do( diff --git a/pkg/consumoor/clickhouse/writer.go b/pkg/consumoor/clickhouse/writer.go index 94a3ed5c..3685f7cb 100644 --- a/pkg/consumoor/clickhouse/writer.go +++ b/pkg/consumoor/clickhouse/writer.go @@ -15,6 +15,32 @@ import ( "github.com/sirupsen/logrus" ) +// FlushResult holds the structured outcome of a FlushTableEvents call. +type FlushResult struct { + // InvalidEvents contains events that failed FlattenTo with + // route.ErrInvalidEvent. These are permanently unflattenable and + // should be sent to the DLQ by the caller. + InvalidEvents []*xatu.DecoratedEvent + + // TableErrors maps base table names to their flush errors. + // Tables that succeeded are absent from the map. + TableErrors map[string]error +} + +// Err returns a joined error of all table failures, or nil if all succeeded. +func (r *FlushResult) Err() error { + if r == nil || len(r.TableErrors) == 0 { + return nil + } + + errs := make([]error, 0, len(r.TableErrors)) + for _, e := range r.TableErrors { + errs = append(errs, e) + } + + return errors.Join(errs...) +} + // ChGoWriter manages batched inserts using the ch-go client. type ChGoWriter struct { log logrus.FieldLogger @@ -186,17 +212,21 @@ func (w *ChGoWriter) Stop(_ context.Context) error { // FlushTableEvents writes the given events directly to their respective // ClickHouse tables concurrently. The map keys are base table names -// (without suffix). Returns a joined error containing all table failures. +// (without suffix). Returns a FlushResult containing per-table errors +// and any invalid events that should be sent to the DLQ. func (w *ChGoWriter) FlushTableEvents( ctx context.Context, tableEvents map[string][]*xatu.DecoratedEvent, -) error { +) *FlushResult { + result := &FlushResult{} + if len(tableEvents) == 0 { - return nil + return result } type tableFlush struct { tw *chTableWriter + base string events []*xatu.DecoratedEvent } @@ -208,14 +238,20 @@ func (w *ChGoWriter) FlushTableEvents( } tw := w.getOrCreateTableWriter(base) - flushes = append(flushes, tableFlush{tw: tw, events: events}) + flushes = append(flushes, tableFlush{tw: tw, base: base, events: events}) } if len(flushes) == 0 { - return nil + return result + } + + type flushOutcome struct { + base string + invalidEvents []*xatu.DecoratedEvent + err error } - errs := make([]error, len(flushes)) + outcomes := make([]flushOutcome, len(flushes)) var wg sync.WaitGroup @@ -225,13 +261,28 @@ func (w *ChGoWriter) FlushTableEvents( go func(idx int, f tableFlush) { defer wg.Done() - errs[idx] = f.tw.flush(ctx, f.events) + invalid, err := f.tw.flush(ctx, f.events) + outcomes[idx] = flushOutcome{base: f.base, invalidEvents: invalid, err: err} }(i, f) } wg.Wait() - return errors.Join(errs...) + for _, o := range outcomes { + if len(o.invalidEvents) > 0 { + result.InvalidEvents = append(result.InvalidEvents, o.invalidEvents...) + } + + if o.err != nil { + if result.TableErrors == nil { + result.TableErrors = make(map[string]error, len(flushes)) + } + + result.TableErrors[o.base] = o.err + } + } + + return result } func (w *ChGoWriter) getOrCreateTableWriter(table string) *chTableWriter { diff --git a/pkg/consumoor/clickhouse/writer_benchmark_test.go b/pkg/consumoor/clickhouse/writer_benchmark_test.go index de0e5abb..32b093e7 100644 --- a/pkg/consumoor/clickhouse/writer_benchmark_test.go +++ b/pkg/consumoor/clickhouse/writer_benchmark_test.go @@ -124,7 +124,7 @@ func BenchmarkFlushTableEvents(b *testing.B) { benchTable: {event}, } - if err := w.FlushTableEvents(context.Background(), tableEvents); err != nil { + if err := w.FlushTableEvents(context.Background(), tableEvents).Err(); err != nil { b.Fatalf("FlushTableEvents: %v", err) } } @@ -154,7 +154,7 @@ func BenchmarkFlushTableEventsBatch(b *testing.B) { benchTable: events, } - if err := w.FlushTableEvents(context.Background(), tableEvents); err != nil { + if err := w.FlushTableEvents(context.Background(), tableEvents).Err(); err != nil { b.Fatalf("FlushTableEvents: %v", err) } } @@ -185,7 +185,7 @@ func BenchmarkFlushConcurrent(b *testing.B) { benchTable: events, } - if err := w.FlushTableEvents(context.Background(), tableEvents); err != nil { + if err := w.FlushTableEvents(context.Background(), tableEvents).Err(); err != nil { b.Logf("FlushTableEvents: %v", err) } } @@ -239,7 +239,7 @@ func BenchmarkEndToEndWithFlatten(b *testing.B) { benchTable: {event}, } - if err := w.FlushTableEvents(context.Background(), tableEvents); err != nil { + if err := w.FlushTableEvents(context.Background(), tableEvents).Err(); err != nil { b.Fatalf("FlushTableEvents: %v", err) } } diff --git a/pkg/consumoor/clickhouse/writer_test.go b/pkg/consumoor/clickhouse/writer_test.go index 7ec0626f..eb9cd946 100644 --- a/pkg/consumoor/clickhouse/writer_test.go +++ b/pkg/consumoor/clickhouse/writer_test.go @@ -137,26 +137,46 @@ func TestIsPermanentWriteError(t *testing.T) { cause: &inputPrepError{cause: errors.New("no batch factory")}, })) + // Data-quality errors are permanent. assert.True(t, IsPermanentWriteError(&tableWriteError{ + table: "beacon_head", + cause: &ch.Exception{ + Code: proto.ErrCannotParseNumber, + Name: "CANNOT_PARSE_NUMBER", + Message: "bad number", + }, + })) + + // Schema mismatches are NOT permanent — they can resolve after migration. + assert.False(t, IsPermanentWriteError(&tableWriteError{ table: "beacon_head", cause: &ch.Exception{ Code: proto.ErrUnknownIdentifier, Name: "UNKNOWN_IDENTIFIER", Message: "unknown column", }, - })) + }), "schema errors should not be permanent (transient during deploys)") + + assert.False(t, IsPermanentWriteError(&tableWriteError{ + table: "beacon_head", + cause: &ch.Exception{ + Code: proto.ErrUnknownTable, + Name: "UNKNOWN_TABLE", + Message: "table not found", + }, + }), "unknown table should not be permanent (transient during deploys)") assert.False(t, IsPermanentWriteError(errors.New("dial tcp timeout"))) } -func TestFlattenErrorIsNotPermanent(t *testing.T) { +func TestFlattenErrorIsPermanent(t *testing.T) { err := &tableWriteError{ table: "beacon_head", cause: &flattenError{cause: errors.New("bad proto field")}, } - assert.False(t, IsPermanentWriteError(err), - "flattenError must NOT be classified as permanent") + assert.True(t, IsPermanentWriteError(err), + "flattenError must be classified as permanent") var flatErr *flattenError assert.True(t, errors.As(err, &flatErr), @@ -179,10 +199,20 @@ func TestIsPermanentWriteErrorJoined(t *testing.T) { cause: errors.New("connection reset"), } - t.Run("joined with permanent sub-error", func(t *testing.T) { + t.Run("mixed permanent and transient is NOT permanent", func(t *testing.T) { joined := errors.Join(permanent, transient) + assert.False(t, IsPermanentWriteError(joined), + "joined error with any transient sub-error should NOT be permanent (NAK for retry)") + }) + + t.Run("all permanent sub-errors is permanent", func(t *testing.T) { + permanent2 := &tableWriteError{ + table: "table_c", + cause: &inputPrepError{cause: errors.New("another schema issue")}, + } + joined := errors.Join(permanent, permanent2) assert.True(t, IsPermanentWriteError(joined), - "joined error containing a permanent sub-error should be permanent") + "joined error with only permanent sub-errors should be permanent") }) t.Run("joined with only transient sub-errors", func(t *testing.T) { @@ -194,6 +224,12 @@ func TestIsPermanentWriteErrorJoined(t *testing.T) { assert.False(t, IsPermanentWriteError(joined), "joined error with only transient sub-errors should not be permanent") }) + + t.Run("joined with nil sub-errors", func(t *testing.T) { + joined := errors.Join(nil, permanent, nil) + assert.True(t, IsPermanentWriteError(joined), + "nil sub-errors should be ignored; remaining permanent error makes it permanent") + }) } // --------------------------------------------------------------------------- @@ -445,7 +481,7 @@ func TestFlushTableEvents_ConcurrentFlush(t *testing.T) { // Launch multiple concurrent FlushTableEvents calls. const flushers = 10 - errs := make([]error, flushers) + results := make([]*FlushResult, flushers) var wg sync.WaitGroup @@ -455,7 +491,7 @@ func TestFlushTableEvents_ConcurrentFlush(t *testing.T) { go func(idx int) { defer wg.Done() - errs[idx] = w.FlushTableEvents(context.Background(), map[string][]*xatu.DecoratedEvent{ + results[idx] = w.FlushTableEvents(context.Background(), map[string][]*xatu.DecoratedEvent{ table: {event}, }) }(i) @@ -463,8 +499,8 @@ func TestFlushTableEvents_ConcurrentFlush(t *testing.T) { wg.Wait() - for i, err := range errs { - assert.NoError(t, err, "flusher %d returned error", i) + for i, result := range results { + assert.NoError(t, result.Err(), "flusher %d returned error", i) } assert.Equal(t, int32(flushers), flushCount.Load(), "each flush should have triggered a pool.Do call") diff --git a/pkg/consumoor/config.go b/pkg/consumoor/config.go index 3f9b0b91..fa45341e 100644 --- a/pkg/consumoor/config.go +++ b/pkg/consumoor/config.go @@ -1,6 +1,7 @@ package consumoor import ( + "errors" "fmt" "sort" "strings" @@ -30,6 +31,10 @@ type Config struct { // Validate checks the configuration for errors. func (c *Config) Validate() error { + if c.MetricsAddr == "" { + return errors.New("metricsAddr is required") + } + if err := c.Kafka.Validate(); err != nil { return err } diff --git a/pkg/consumoor/config_test.go b/pkg/consumoor/config_test.go index 4989a7e1..342c9535 100644 --- a/pkg/consumoor/config_test.go +++ b/pkg/consumoor/config_test.go @@ -40,16 +40,20 @@ func validClickHouseConfig() *clickhouse.Config { // validKafkaConfig returns a KafkaConfig that passes validation. func validKafkaConfig() *source.KafkaConfig { return &source.KafkaConfig{ - Brokers: []string{"localhost:9092"}, - Topics: []string{"^test-.+"}, - ConsumerGroup: "test-group", - Encoding: "json", - OffsetDefault: "earliest", - SessionTimeoutMs: 30000, - RebalanceTimeout: 15 * time.Second, - CommitInterval: 5 * time.Second, - ShutdownTimeout: 30 * time.Second, - MaxInFlight: 64, + Brokers: []string{"localhost:9092"}, + Topics: []string{"^test-.+"}, + ConsumerGroup: "test-group", + Encoding: "json", + OffsetDefault: "earliest", + SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, + CommitInterval: 5 * time.Second, + ShutdownTimeout: 30 * time.Second, + MaxInFlight: 64, + FetchMinBytes: 1, + FetchWaitMaxMs: 250, + MaxPartitionFetchBytes: 3145728, + FetchMaxBytes: 10485760, } } diff --git a/pkg/consumoor/consumoor.go b/pkg/consumoor/consumoor.go index ecf20e68..4eb2a1fd 100644 --- a/pkg/consumoor/consumoor.go +++ b/pkg/consumoor/consumoor.go @@ -9,6 +9,7 @@ import ( "net/http" "os/signal" "regexp" + "strings" "sync" "syscall" "time" @@ -38,8 +39,9 @@ type topicStream struct { // Benthos stream and consumer group while sharing a single ClickHouse // writer for efficient connection reuse. type Consumoor struct { - log logrus.FieldLogger - config *Config + log logrus.FieldLogger + parentLog logrus.FieldLogger + config *Config metrics *telemetry.Metrics router *router.Engine @@ -47,6 +49,10 @@ type Consumoor struct { streams []topicStream lagMonitor *source.LagMonitor + // activeTopics tracks topics that have a running stream to avoid + // creating duplicate streams when watchTopics re-discovers them. + activeTopics map[string]struct{} + mu sync.Mutex metricsServer *http.Server pprofServer *http.Server @@ -138,6 +144,11 @@ func New( rtr, writer, false, // writer lifecycle owned by Consumoor, not the output plugin + source.GroupRetryConfig{ + MaxAttempts: config.ClickHouse.ChGo.GroupRetryMaxAttempts, + BaseDelay: config.ClickHouse.ChGo.GroupRetryBaseDelay, + MaxDelay: config.ClickHouse.ChGo.GroupRetryMaxDelay, + }, ) if sErr != nil { return nil, fmt.Errorf( @@ -149,17 +160,37 @@ func New( topic: topic, stream: stream, }) + + metrics.OutputMaxInFlight().WithLabelValues(topic).Set(float64(topicKafkaCfg.MaxInFlight)) + } + + activeTopics := make(map[string]struct{}, len(topics)) + for _, t := range topics { + activeTopics[t] = struct{}{} + } + + if strings.TrimSpace(config.Kafka.RejectedTopic) == "" { + cLog.Warn("No rejectedTopic configured — decode errors and permanent write failures will block Kafka offset advancement (messages will be redelivered indefinitely). Configure kafka.rejectedTopic to enable dead letter queue") } - metrics.OutputMaxInFlight().Set(float64(config.Kafka.MaxInFlight)) + if len(config.Kafka.TopicOverrides) > 0 { + for overrideTopic := range config.Kafka.TopicOverrides { + if _, found := activeTopics[overrideTopic]; !found { + cLog.WithField("topic", overrideTopic). + Warn("Topic override configured but no matching topic was discovered — check for typos") + } + } + } c := &Consumoor{ - log: cLog, - config: config, - metrics: metrics, - router: rtr, - writer: writer, - streams: streams, + log: cLog, + parentLog: log, + config: config, + metrics: metrics, + router: rtr, + writer: writer, + streams: streams, + activeTopics: activeTopics, } // Optionally create the Kafka consumer lag monitor. @@ -186,7 +217,7 @@ func (c *Consumoor) Start(ctx context.Context) error { nctx, stop := signal.NotifyContext(ctx, syscall.SIGINT, syscall.SIGTERM) defer stop() - if err := c.writer.Start(ctx); err != nil { + if err := c.writer.Start(nctx); err != nil { return fmt.Errorf("starting clickhouse writer: %w", err) } @@ -235,7 +266,7 @@ func (c *Consumoor) Start(ctx context.Context) error { if c.config.Kafka.TopicRefreshInterval > 0 { g.Go(func() error { - c.watchTopics(gCtx) + c.watchTopics(gCtx, g) return nil }) @@ -246,7 +277,7 @@ func (c *Consumoor) Start(ctx context.Context) error { // All streams and HTTP servers have exited. Now stop the writer. c.stopWriter(ctx) - if streamErr != nil && streamErr != context.Canceled { + if streamErr != nil && !errors.Is(streamErr, context.Canceled) { return streamErr } @@ -255,20 +286,23 @@ func (c *Consumoor) Start(ctx context.Context) error { // stopHTTPServers shuts down the metrics and pprof servers. Called from // within the errgroup on context cancellation so that g.Wait() can return. -func (c *Consumoor) stopHTTPServers(ctx context.Context) { +func (c *Consumoor) stopHTTPServers(_ context.Context) { c.mu.Lock() metricsServer := c.metricsServer pprofServer := c.pprofServer c.mu.Unlock() + shutdownCtx, cancel := context.WithTimeout(context.Background(), 15*time.Second) + defer cancel() + if metricsServer != nil { - if err := metricsServer.Shutdown(ctx); err != nil { + if err := metricsServer.Shutdown(shutdownCtx); err != nil { c.log.WithError(err).Error("Error stopping metrics server") } } if pprofServer != nil { - if err := pprofServer.Shutdown(ctx); err != nil { + if err := pprofServer.Shutdown(shutdownCtx); err != nil { c.log.WithError(err).Error("Error stopping pprof server") } } @@ -353,17 +387,16 @@ func (c *Consumoor) startMetrics(ctx context.Context) error { } // watchTopics periodically queries Kafka metadata to discover topics matching -// the configured regex patterns. It logs newly appeared and disappeared topics -// and updates the active_topics gauge. The actual consumption of new topics is -// handled by Benthos via metadata_max_age; this goroutine provides visibility. -func (c *Consumoor) watchTopics(ctx context.Context) { +// the configured regex patterns. Newly discovered topics automatically get +// their own Benthos stream without requiring a restart. +func (c *Consumoor) watchTopics(ctx context.Context, g *errgroup.Group) { interval := c.config.Kafka.TopicRefreshInterval c.log.WithField("interval", interval). Info("Starting topic discovery watcher") // Perform an initial discovery so the metric is populated immediately. - knownTopics := c.discoverAndDiff(ctx, nil) + knownTopics := c.discoverAndDiff(ctx, g, nil) ticker := time.NewTicker(interval) defer ticker.Stop() @@ -375,16 +408,17 @@ func (c *Consumoor) watchTopics(ctx context.Context) { return case <-ticker.C: - knownTopics = c.discoverAndDiff(ctx, knownTopics) + knownTopics = c.discoverAndDiff(ctx, g, knownTopics) } } } // discoverAndDiff calls DiscoverTopics, compares the result to the previously -// known set, logs any changes, and updates the active_topics gauge. It returns -// the current set of discovered topics for the next comparison cycle. +// known set, logs any changes, updates the active_topics gauge, and starts +// streams for newly discovered topics. func (c *Consumoor) discoverAndDiff( ctx context.Context, + g *errgroup.Group, previous map[string]struct{}, ) map[string]struct{} { topics, err := source.DiscoverTopics(ctx, &c.config.Kafka) @@ -409,11 +443,12 @@ func (c *Consumoor) discoverAndDiff( return current } - // Find newly appeared topics. + // Find newly appeared topics and start streams for them. for _, t := range topics { if _, ok := previous[t]; !ok { c.log.WithField("topic", t). Info("Discovered new topic matching pattern") + c.startTopicStream(ctx, g, t) } } @@ -428,6 +463,85 @@ func (c *Consumoor) discoverAndDiff( return current } +// startTopicStream creates a Benthos stream for a dynamically discovered +// topic and launches it in the errgroup. If the topic already has an active +// stream, this is a no-op. +func (c *Consumoor) startTopicStream( + ctx context.Context, + g *errgroup.Group, + topic string, +) { + c.mu.Lock() + if _, exists := c.activeTopics[topic]; exists { + c.mu.Unlock() + + return + } + + c.activeTopics[topic] = struct{}{} + c.mu.Unlock() + + topicKafkaCfg := c.config.Kafka.ApplyTopicOverride(topic) + topicKafkaCfg.Topics = []string{"^" + regexp.QuoteMeta(topic) + "$"} + topicKafkaCfg.ConsumerGroup = c.config.Kafka.ConsumerGroup + "-" + topic + + if _, hasOverride := c.config.Kafka.TopicOverrides[topic]; hasOverride { + c.log.WithField("topic", topic). + WithField("outputBatchCount", topicKafkaCfg.OutputBatchCount). + WithField("outputBatchPeriod", topicKafkaCfg.OutputBatchPeriod). + WithField("maxInFlight", topicKafkaCfg.MaxInFlight). + Info("Applied per-topic batch overrides for dynamically discovered topic") + } + + stream, err := source.NewBenthosStream( + c.parentLog.WithField("topic", topic), + c.config.LoggingLevel, + &topicKafkaCfg, + c.metrics, + c.router, + c.writer, + false, // writer lifecycle owned by Consumoor + source.GroupRetryConfig{ + MaxAttempts: c.config.ClickHouse.ChGo.GroupRetryMaxAttempts, + BaseDelay: c.config.ClickHouse.ChGo.GroupRetryBaseDelay, + MaxDelay: c.config.ClickHouse.ChGo.GroupRetryMaxDelay, + }, + ) + if err != nil { + c.log.WithError(err). + WithField("topic", topic). + Error("Failed to create stream for dynamically discovered topic") + + c.mu.Lock() + delete(c.activeTopics, topic) + c.mu.Unlock() + + return + } + + c.mu.Lock() + c.streams = append(c.streams, topicStream{topic: topic, stream: stream}) + c.mu.Unlock() + + c.metrics.OutputMaxInFlight().WithLabelValues(topic).Set(float64(topicKafkaCfg.MaxInFlight)) + + c.log.WithField("topic", topic). + Info("Starting stream for dynamically discovered topic") + + if c.lagMonitor != nil { + c.log.WithField("topic", topic). + Warn("Dynamically discovered topic will not have consumer lag monitoring — restart to include it") + } + + g.Go(func() error { + if err := stream.Run(ctx); err != nil && !errors.Is(err, context.Canceled) { + return fmt.Errorf("running stream for topic %q: %w", topic, err) + } + + return nil + }) +} + func (c *Consumoor) startPProf(_ context.Context) error { c.log.WithField("addr", c.config.PProfAddr).Info("Starting pprof server") diff --git a/pkg/consumoor/health_test.go b/pkg/consumoor/health_test.go index c06c3ca9..7ef7785b 100644 --- a/pkg/consumoor/health_test.go +++ b/pkg/consumoor/health_test.go @@ -8,6 +8,7 @@ import ( "net/http/httptest" "testing" + "github.com/ethpandaops/xatu/pkg/consumoor/clickhouse" "github.com/ethpandaops/xatu/pkg/proto/xatu" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" @@ -21,8 +22,8 @@ type mockWriter struct { func (m *mockWriter) Start(_ context.Context) error { return nil } func (m *mockWriter) Stop(_ context.Context) error { return nil } -func (m *mockWriter) FlushTableEvents(_ context.Context, _ map[string][]*xatu.DecoratedEvent) error { - return nil +func (m *mockWriter) FlushTableEvents(_ context.Context, _ map[string][]*xatu.DecoratedEvent) *clickhouse.FlushResult { + return &clickhouse.FlushResult{} } func (m *mockWriter) Ping(_ context.Context) error { return m.pingErr } diff --git a/pkg/consumoor/source/benthos.go b/pkg/consumoor/source/benthos.go index 65e73789..bf31cf7a 100644 --- a/pkg/consumoor/source/benthos.go +++ b/pkg/consumoor/source/benthos.go @@ -18,6 +18,16 @@ import ( const benthosOutputType = "xatu_clickhouse" +// GroupRetryConfig configures group-level retry behavior for partial table +// failures in processGroup. When a fanout flush partially fails (some tables +// succeed, others fail transiently), only the failed tables are retried with +// exponential backoff to limit write amplification. +type GroupRetryConfig struct { + MaxAttempts int + BaseDelay time.Duration + MaxDelay time.Duration +} + // NewBenthosStream creates a Benthos stream that consumes from Kafka and writes // to ClickHouse via the custom xatu_clickhouse output plugin. // When ownsWriter is true the output plugin manages the writer lifecycle @@ -31,6 +41,7 @@ func NewBenthosStream( routeEngine *router.Engine, writer Writer, ownsWriter bool, + groupRetry GroupRetryConfig, ) (*service.Stream, error) { if kafkaConfig == nil { return nil, fmt.Errorf("nil kafka config") @@ -63,15 +74,18 @@ func NewBenthosStream( service.NewConfigSpec(), func(_ *service.ParsedConfig, _ *service.Resources) (out service.BatchOutput, policy service.BatchPolicy, maxInFlight int, err error) { return &xatuClickHouseOutput{ - log: log.WithField("component", "benthos_clickhouse_output"), - encoding: kafkaConfig.Encoding, - router: routeEngine, - writer: writer, - metrics: metrics, - rejectSink: rejectSink, - ownsWriter: ownsWriter, - outputBatchCount: kafkaConfig.OutputBatchCount, - logSampler: telemetry.NewLogSampler(30 * time.Second), + log: log.WithField("component", "benthos_clickhouse_output"), + encoding: kafkaConfig.Encoding, + router: routeEngine, + writer: writer, + metrics: metrics, + rejectSink: rejectSink, + ownsWriter: ownsWriter, + outputBatchCount: kafkaConfig.OutputBatchCount, + logSampler: telemetry.NewLogSampler(30 * time.Second), + groupRetryMaxAttempts: groupRetry.MaxAttempts, + groupRetryBaseDelay: groupRetry.BaseDelay, + groupRetryMaxDelay: groupRetry.MaxDelay, }, batchPolicy, kafkaConfig.MaxInFlight, nil }, ); registerErr != nil { diff --git a/pkg/consumoor/source/benthos_test.go b/pkg/consumoor/source/benthos_test.go index 03cc7339..02fb8a08 100644 --- a/pkg/consumoor/source/benthos_test.go +++ b/pkg/consumoor/source/benthos_test.go @@ -23,6 +23,7 @@ import ( ch "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/proto" + "github.com/ethpandaops/xatu/pkg/consumoor/clickhouse" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/consumoor/router" "github.com/ethpandaops/xatu/pkg/consumoor/telemetry" @@ -66,7 +67,7 @@ func (w *testWriter) Stop(context.Context) error { return nil } -func (w *testWriter) FlushTableEvents(_ context.Context, tableEvents map[string][]*xatu.DecoratedEvent) error { +func (w *testWriter) FlushTableEvents(_ context.Context, tableEvents map[string][]*xatu.DecoratedEvent) *clickhouse.FlushResult { w.flushCalls++ if w.writes == nil { @@ -86,10 +87,20 @@ func (w *testWriter) FlushTableEvents(_ context.Context, tableEvents map[string] w.flushErrs = append(w.flushErrs[:i], w.flushErrs[i+1:]...) - return err + result := &clickhouse.FlushResult{ + TableErrors: make(map[string]error, 1), + } + // Attribute the error to the first table in the map for test purposes. + for table := range tableEvents { + result.TableErrors[table] = err + + break + } + + return result } - return nil + return &clickhouse.FlushResult{} } func (w *testWriter) Ping(context.Context) error { diff --git a/pkg/consumoor/source/config.go b/pkg/consumoor/source/config.go index 2c850c79..5fc4d052 100644 --- a/pkg/consumoor/source/config.go +++ b/pkg/consumoor/source/config.go @@ -125,7 +125,7 @@ type KafkaConfig struct { // MaxInFlight is the maximum number of concurrent WriteBatch calls // Benthos makes for each stream's output. Higher values increase // throughput by allowing concurrent ClickHouse INSERTs and bigger - // natural batches. Default: 8. + // natural batches. Default: 64. MaxInFlight int `yaml:"maxInFlight" default:"64"` // TopicOverrides contains per-topic batch settings keyed by exact topic name. @@ -202,6 +202,30 @@ func (c *KafkaConfig) Validate() error { return errors.New("kafka: maxInFlight must be >= 1") } + if c.FetchMinBytes < 1 { + return errors.New("kafka: fetchMinBytes must be >= 1") + } + + if c.FetchWaitMaxMs <= 0 { + return errors.New("kafka: fetchWaitMaxMs must be > 0") + } + + if c.MaxPartitionFetchBytes < 1 { + return errors.New("kafka: maxPartitionFetchBytes must be >= 1") + } + + if c.FetchMaxBytes < 1 { + return errors.New("kafka: fetchMaxBytes must be >= 1") + } + + if c.LagPollInterval < 0 { + return errors.New("kafka: lagPollInterval must be >= 0") + } + + if c.ConnectTimeout < 0 { + return errors.New("kafka: connectTimeout must be >= 0") + } + for topic, override := range c.TopicOverrides { if err := override.Validate(topic); err != nil { return err @@ -265,10 +289,6 @@ func (c *SASLConfig) Validate() error { } } - if c.User == "" { - return errors.New("kafka.sasl: user is required") - } - if c.Password == "" && c.PasswordFile == "" { return errors.New("kafka.sasl: password or passwordFile is required") } @@ -277,5 +297,14 @@ func (c *SASLConfig) Validate() error { return errors.New("kafka.sasl: only one of password or passwordFile can be set") } + // OAUTHBEARER uses a token only; no username is required. + if mechanism == SASLMechanismOAUTHBEARER { + return nil + } + + if c.User == "" { + return errors.New("kafka.sasl: user is required") + } + return nil } diff --git a/pkg/consumoor/source/config_test.go b/pkg/consumoor/source/config_test.go index eca84526..b111a623 100644 --- a/pkg/consumoor/source/config_test.go +++ b/pkg/consumoor/source/config_test.go @@ -10,16 +10,20 @@ import ( func validKafkaConfig() KafkaConfig { return KafkaConfig{ - Brokers: []string{"localhost:9092"}, - Topics: []string{"test-topic"}, - ConsumerGroup: "test-group", - Encoding: "json", - OffsetDefault: "earliest", - SessionTimeoutMs: 30000, - RebalanceTimeout: 15 * time.Second, - CommitInterval: 5 * time.Second, - ShutdownTimeout: 30 * time.Second, - MaxInFlight: 64, + Brokers: []string{"localhost:9092"}, + Topics: []string{"test-topic"}, + ConsumerGroup: "test-group", + Encoding: "json", + OffsetDefault: "earliest", + SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, + CommitInterval: 5 * time.Second, + ShutdownTimeout: 30 * time.Second, + MaxInFlight: 64, + FetchMinBytes: 1, + FetchWaitMaxMs: 250, + MaxPartitionFetchBytes: 3145728, + FetchMaxBytes: 10485760, } } diff --git a/pkg/consumoor/source/dlq.go b/pkg/consumoor/source/dlq.go index 2c11e5ad..9d28623b 100644 --- a/pkg/consumoor/source/dlq.go +++ b/pkg/consumoor/source/dlq.go @@ -31,20 +31,6 @@ type rejectSink interface { Enabled() bool } -type noopRejectSink struct{} - -func (noopRejectSink) Write(_ context.Context, _ *rejectedRecord) error { - return nil -} - -func (noopRejectSink) Close() error { - return nil -} - -func (noopRejectSink) Enabled() bool { - return false -} - type kafkaRejectSink struct { topic string cl *kgo.Client @@ -102,11 +88,11 @@ func (s *kafkaRejectSink) Write(ctx context.Context, record *rejectedRecord) err func newRejectSink(cfg *KafkaConfig) (rejectSink, error) { if cfg == nil { - return noopRejectSink{}, nil + return nil, nil } if strings.TrimSpace(cfg.RejectedTopic) == "" { - return noopRejectSink{}, nil + return nil, nil } opts := []kgo.Opt{ diff --git a/pkg/consumoor/source/kafka.go b/pkg/consumoor/source/kafka.go index e85201fd..a14dff82 100644 --- a/pkg/consumoor/source/kafka.go +++ b/pkg/consumoor/source/kafka.go @@ -99,10 +99,14 @@ func decodeDecoratedEvent(encoding string, data []byte) (*xatu.DecoratedEvent, e switch encoding { case "protobuf": if err := event.UnmarshalVT(data); err != nil { + event.ReturnToVTPool() + return nil, fmt.Errorf("protobuf unmarshal: %w", err) } default: if err := jsonUnmarshalOpts.Unmarshal(data, event); err != nil { + event.ReturnToVTPool() + return nil, fmt.Errorf("json unmarshal: %w", err) } } diff --git a/pkg/consumoor/source/lag_monitor.go b/pkg/consumoor/source/lag_monitor.go index df9f0103..36532d3c 100644 --- a/pkg/consumoor/source/lag_monitor.go +++ b/pkg/consumoor/source/lag_monitor.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "strconv" + "sync" "time" "github.com/twmb/franz-go/pkg/kadm" @@ -27,6 +28,7 @@ type LagMonitor struct { kgoClient *kgo.Client done chan struct{} exited chan struct{} + stopOnce sync.Once } // NewLagMonitor creates a new LagMonitor. Call Start to begin polling. @@ -109,11 +111,14 @@ func (m *LagMonitor) Start(ctx context.Context) error { } // Stop signals the lag monitor to exit and waits for it to finish. +// It is safe to call multiple times; only the first call performs cleanup. func (m *LagMonitor) Stop() error { - close(m.done) - <-m.exited + m.stopOnce.Do(func() { + close(m.done) + <-m.exited - m.kgoClient.Close() + m.kgoClient.Close() + }) return nil } diff --git a/pkg/consumoor/source/output.go b/pkg/consumoor/source/output.go index 54d74146..0587809a 100644 --- a/pkg/consumoor/source/output.go +++ b/pkg/consumoor/source/output.go @@ -5,6 +5,7 @@ import ( "errors" "fmt" "sync" + "time" "github.com/redpanda-data/benthos/v4/public/service" @@ -19,6 +20,7 @@ const ( rejectReasonDecode = "decode_error" rejectReasonRouteRejected = "route_rejected" rejectReasonWritePermanent = "write_permanent" + rejectReasonInvalidEvent = "invalid_event" ) var errBatchWriteFailed = errors.New("clickhouse batch write failed") @@ -48,6 +50,11 @@ type xatuClickHouseOutput struct { outputBatchCount int logSampler *telemetry.LogSampler + // Group-level retry config for partial table failures. + groupRetryMaxAttempts int + groupRetryBaseDelay time.Duration + groupRetryMaxDelay time.Duration + mu sync.Mutex started bool } @@ -274,12 +281,10 @@ func (o *xatuClickHouseOutput) Close(ctx context.Context) error { // The caller's accumulated batchErr is threaded through so that failures from // earlier phases (e.g. decode errors) are preserved. // -// NOTE: for events that fan out to multiple tables, a partial failure (some -// tables succeed, others fail) will NAK the entire group. On redelivery the -// successfully written tables will receive duplicate rows. This is inherent -// to the at-least-once delivery model; ClickHouse deduplication (e.g. -// ReplacingMergeTree) should be used for tables that require exactly-once -// semantics. +// For events that fan out to multiple tables, partial failures (some tables +// succeed, others fail transiently) are retried with exponential backoff. +// Only the failed tables are retried to prevent duplicate writes on already- +// succeeded tables. func (o *xatuClickHouseOutput) processGroup( ctx context.Context, msgs service.MessageBatch, @@ -293,7 +298,85 @@ func (o *xatuClickHouseOutput) processGroup( } } - err := o.writer.FlushTableEvents(ctx, tableEvents) + var ( + allInvalidEvents []*xatu.DecoratedEvent + lastResult *clickhouse.FlushResult + ) + + // Group-level retry loop for partial table failures. + for attempt := range o.groupRetryMaxAttempts + 1 { + if attempt > 0 && len(tableEvents) > 0 { + delay := min( + o.groupRetryBaseDelay*time.Duration(1<<(attempt-1)), + o.groupRetryMaxDelay, + ) + + o.log.WithFields(logrus.Fields{ + "attempt": attempt, + "max_attempts": o.groupRetryMaxAttempts, + "delay": delay, + "failed_tables": len(tableEvents), + }).Warn("Retrying failed tables with backoff") + + o.metrics.GroupRetries().WithLabelValues( + g.messages[0].event.GetEvent().GetName().String(), + ).Inc() + + select { + case <-ctx.Done(): + for _, gm := range g.messages { + batchErr = addBatchFailure(batchErr, msgs, gm.batchIndex, ctx.Err()) + } + + return batchErr + case <-time.After(delay): + } + } + + result := o.writer.FlushTableEvents(ctx, tableEvents) + lastResult = result + + if len(result.InvalidEvents) > 0 { + allInvalidEvents = append(allInvalidEvents, result.InvalidEvents...) + } + + // All tables succeeded. + if len(result.TableErrors) == 0 { + break + } + + // Check if any failures are permanent — don't retry those. + anyTransient := false + + for _, tErr := range result.TableErrors { + if !clickhouse.IsPermanentWriteError(tErr) { + anyTransient = true + + break + } + } + + if !anyTransient { + break + } + + // Remove succeeded tables from the retry set. + remaining := make(map[string][]*xatu.DecoratedEvent, len(result.TableErrors)) + + for table := range result.TableErrors { + if events, ok := tableEvents[table]; ok { + remaining[table] = events + } + } + + tableEvents = remaining + } + + // DLQ invalid events individually. Deduplicate by event pointer since + // the same event can appear in multiple tables' invalid lists due to fanout. + batchErr = o.dlqInvalidEvents(ctx, msgs, batchErr, g, allInvalidEvents) + + err := lastResult.Err() if err == nil { return batchErr } @@ -331,6 +414,51 @@ func (o *xatuClickHouseOutput) processGroup( return batchErr } +// dlqInvalidEvents sends invalid events to the DLQ, deduplicating by event +// pointer to handle fanout where the same event appears in multiple tables. +func (o *xatuClickHouseOutput) dlqInvalidEvents( + ctx context.Context, + msgs service.MessageBatch, + batchErr *service.BatchError, + g *eventGroup, + invalidEvents []*xatu.DecoratedEvent, +) *service.BatchError { + if len(invalidEvents) == 0 { + return batchErr + } + + seen := make(map[*xatu.DecoratedEvent]struct{}, len(invalidEvents)) + + for _, ev := range invalidEvents { + if _, dup := seen[ev]; dup { + continue + } + + seen[ev] = struct{}{} + + for _, gm := range g.messages { + if gm.event != ev { + continue + } + + rejectErr := o.rejectMessage(ctx, &rejectedRecord{ + Reason: rejectReasonInvalidEvent, + Err: "invalid event: permanently unflattenable", + Payload: append([]byte(nil), gm.raw...), + EventName: ev.GetEvent().GetName().String(), + Kafka: gm.kafka, + }) + if rejectErr != nil { + batchErr = addBatchFailure(batchErr, msgs, gm.batchIndex, rejectErr) + } + + break + } + } + + return batchErr +} + func (o *xatuClickHouseOutput) rejectMessage( ctx context.Context, record *rejectedRecord, @@ -348,9 +476,11 @@ func (o *xatuClickHouseOutput) rejectMessage( return nil } - // For all other reasons (decode errors, permanent write failures) - // failing the message forces Kafka to redeliver rather than - // silently dropping data when no DLQ is configured. + // For all other reasons (decode errors, permanent write failures, + // invalid events) failing the message forces Kafka to redeliver + // rather than silently dropping data when no DLQ is configured. + // Invalid events may become valid after a rolling deploy where a + // newer version adds support for new fields or event subtypes. return fmt.Errorf("no DLQ configured for rejected message (%s): %s", record.Reason, record.Err) } diff --git a/pkg/consumoor/source/writer.go b/pkg/consumoor/source/writer.go index c89a0874..547a1085 100644 --- a/pkg/consumoor/source/writer.go +++ b/pkg/consumoor/source/writer.go @@ -3,6 +3,7 @@ package source import ( "context" + "github.com/ethpandaops/xatu/pkg/consumoor/clickhouse" "github.com/ethpandaops/xatu/pkg/proto/xatu" ) @@ -12,8 +13,9 @@ type Writer interface { Stop(ctx context.Context) error // FlushTableEvents writes the given events directly to their respective // ClickHouse tables concurrently. The map keys are base table names - // (without suffix). Returns a joined error containing all table failures. - FlushTableEvents(ctx context.Context, tableEvents map[string][]*xatu.DecoratedEvent) error + // (without suffix). Returns a FlushResult containing per-table errors + // and any invalid events that should be sent to the DLQ. + FlushTableEvents(ctx context.Context, tableEvents map[string][]*xatu.DecoratedEvent) *clickhouse.FlushResult // Ping checks connectivity to the underlying datastore. Ping(ctx context.Context) error } diff --git a/pkg/consumoor/telemetry/metrics.go b/pkg/consumoor/telemetry/metrics.go index 76ebd017..458ff4ee 100644 --- a/pkg/consumoor/telemetry/metrics.go +++ b/pkg/consumoor/telemetry/metrics.go @@ -23,8 +23,9 @@ type Metrics struct { batchFlushTrigger *prometheus.CounterVec activeTopics prometheus.Gauge kafkaConsumerLag *prometheus.GaugeVec - outputMaxInFlight prometheus.Gauge + outputMaxInFlight *prometheus.GaugeVec writeRetries *prometheus.CounterVec + groupRetries *prometheus.CounterVec // adaptive limiter metrics (per-table) adaptiveLimiterLimit *prometheus.GaugeVec @@ -145,7 +146,7 @@ func NewMetrics(namespace string) *Metrics { Subsystem: subsystem, Name: "event_lag_seconds", Help: "Lag between the event timestamp and the wall clock time at routing.", - Buckets: []float64{0.1, 0.5, 1, 2, 5, 10, 30, 60, 120, 300, 600}, + Buckets: []float64{1, 5, 10, 30, 60, 120, 300, 600, 1800, 3600, 7200, 14400}, }, []string{"event_name"}), batchFlushTrigger: promauto.NewCounterVec(prometheus.CounterOpts{ @@ -169,12 +170,12 @@ func NewMetrics(namespace string) *Metrics { Help: "Kafka consumer group lag per topic and partition.", }, []string{"topic", "partition", "consumer_group"}), - outputMaxInFlight: promauto.NewGauge(prometheus.GaugeOpts{ + outputMaxInFlight: promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, Name: "output_max_in_flight", Help: "Configured maximum number of concurrent Benthos WriteBatch calls per stream.", - }), + }, []string{"topic"}), adaptiveLimiterLimit: promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: namespace, @@ -211,6 +212,13 @@ func NewMetrics(namespace string) *Metrics { Help: "Total number of ch-go operation retry attempts after transient errors.", }, []string{"operation"}), + groupRetries: promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "group_retries_total", + Help: "Total number of group-level retry attempts for partial table failures.", + }, []string{"event_name"}), + chgoPoolAcquiredResources: promauto.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, @@ -299,7 +307,7 @@ func (m *Metrics) EventLag() *prometheus.HistogramVec { return m.eventLag func (m *Metrics) BatchFlushTrigger() *prometheus.CounterVec { return m.batchFlushTrigger } func (m *Metrics) ActiveTopics() prometheus.Gauge { return m.activeTopics } func (m *Metrics) KafkaConsumerLag() *prometheus.GaugeVec { return m.kafkaConsumerLag } -func (m *Metrics) OutputMaxInFlight() prometheus.Gauge { return m.outputMaxInFlight } +func (m *Metrics) OutputMaxInFlight() *prometheus.GaugeVec { return m.outputMaxInFlight } func (m *Metrics) AdaptiveLimiterLimit() *prometheus.GaugeVec { return m.adaptiveLimiterLimit } func (m *Metrics) AdaptiveLimiterInflight() *prometheus.GaugeVec { return m.adaptiveLimiterInflight } @@ -308,6 +316,7 @@ func (m *Metrics) AdaptiveLimiterRejections() *prometheus.CounterVec { return m.adaptiveLimiterRejections } func (m *Metrics) WriteRetries() *prometheus.CounterVec { return m.writeRetries } +func (m *Metrics) GroupRetries() *prometheus.CounterVec { return m.groupRetries } func (m *Metrics) ChgoPoolAcquiredResources() prometheus.Gauge { return m.chgoPoolAcquiredResources } func (m *Metrics) ChgoPoolIdleResources() prometheus.Gauge { return m.chgoPoolIdleResources } From d966f0e7121d19acff5ae2ba2f65d7e09afb9130 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Mon, 2 Mar 2026 15:33:29 +1000 Subject: [PATCH 20/27] consumoor: drop intentionally unsupported events; NAK unknown routes --- pkg/consumoor/route/routes_test.go | 18 ++-------- pkg/consumoor/route/unsupported_events.go | 39 +++++++++++++++++++++ pkg/consumoor/router/engine.go | 28 ++++++++++++--- pkg/consumoor/router/engine_test.go | 42 +++++++++++++++++++++++ pkg/consumoor/source/benthos_test.go | 36 +++++++++++++++---- 5 files changed, 137 insertions(+), 26 deletions(-) create mode 100644 pkg/consumoor/route/unsupported_events.go diff --git a/pkg/consumoor/route/routes_test.go b/pkg/consumoor/route/routes_test.go index 0fa57553..3180e78c 100644 --- a/pkg/consumoor/route/routes_test.go +++ b/pkg/consumoor/route/routes_test.go @@ -20,22 +20,8 @@ import ( ) func TestRegistryCoversAllKnownEvents(t *testing.T) { - // Explicitly document events consumoor intentionally does not flatten yet. - unsupported := map[xatu.Event_Name]string{ - xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE: "debug stream not modeled as table output", - xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE_V2: "debug stream not modeled as table output", - xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE_REORG: "debug stream not modeled as table output", - xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE_REORG_V2: "debug stream not modeled as table output", - xatu.Event_BEACON_API_ETH_V1_EVENTS_ATTESTATION: "deprecated in favor of V2 event", - xatu.Event_BEACON_API_ETH_V1_EVENTS_BLOCK: "deprecated in favor of V2 event", - xatu.Event_BEACON_API_ETH_V1_EVENTS_CHAIN_REORG: "deprecated in favor of V2 event", - xatu.Event_BEACON_API_ETH_V1_EVENTS_CONTRIBUTION_AND_PROOF: "deprecated in favor of V2 event", - xatu.Event_BEACON_API_ETH_V1_EVENTS_FINALIZED_CHECKPOINT: "deprecated in favor of V2 event", - xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD: "deprecated in favor of V2 event", - xatu.Event_BEACON_API_ETH_V1_EVENTS_VOLUNTARY_EXIT: "deprecated in favor of V2 event", - xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK: "deprecated in favor of V2 event", - xatu.Event_BEACON_P2P_ATTESTATION: "legacy event path not consumed by consumoor", - } + // Explicitly documented events consumoor intentionally does not flatten. + unsupported := route.UnsupportedEvents() covered := make(map[xatu.Event_Name]struct{}, 128) diff --git a/pkg/consumoor/route/unsupported_events.go b/pkg/consumoor/route/unsupported_events.go new file mode 100644 index 00000000..c1156b22 --- /dev/null +++ b/pkg/consumoor/route/unsupported_events.go @@ -0,0 +1,39 @@ +package route + +import "github.com/ethpandaops/xatu/pkg/proto/xatu" + +// intentionallyUnsupportedEvents documents event types consumoor deliberately +// does not flatten into ClickHouse tables. +var intentionallyUnsupportedEvents = map[xatu.Event_Name]string{ + xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE: "debug stream not modeled as table output", + xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE_V2: "debug stream not modeled as table output", + xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE_REORG: "debug stream not modeled as table output", + xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE_REORG_V2: "debug stream not modeled as table output", + xatu.Event_BEACON_API_ETH_V1_EVENTS_ATTESTATION: "deprecated in favor of V2 event", + xatu.Event_BEACON_API_ETH_V1_EVENTS_BLOCK: "deprecated in favor of V2 event", + xatu.Event_BEACON_API_ETH_V1_EVENTS_CHAIN_REORG: "deprecated in favor of V2 event", + xatu.Event_BEACON_API_ETH_V1_EVENTS_CONTRIBUTION_AND_PROOF: "deprecated in favor of V2 event", + xatu.Event_BEACON_API_ETH_V1_EVENTS_FINALIZED_CHECKPOINT: "deprecated in favor of V2 event", + xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD: "deprecated in favor of V2 event", + xatu.Event_BEACON_API_ETH_V1_EVENTS_VOLUNTARY_EXIT: "deprecated in favor of V2 event", + xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK: "deprecated in favor of V2 event", + xatu.Event_BEACON_P2P_ATTESTATION: "legacy event path not consumed by consumoor", +} + +// UnsupportedReason returns the documented reason when an event is +// intentionally unsupported by consumoor flatteners. +func UnsupportedReason(event xatu.Event_Name) (string, bool) { + reason, ok := intentionallyUnsupportedEvents[event] + + return reason, ok +} + +// UnsupportedEvents returns a copy of intentionally unsupported event reasons. +func UnsupportedEvents() map[xatu.Event_Name]string { + out := make(map[xatu.Event_Name]string, len(intentionallyUnsupportedEvents)) + for event, reason := range intentionallyUnsupportedEvents { + out[event] = reason + } + + return out +} diff --git a/pkg/consumoor/router/engine.go b/pkg/consumoor/router/engine.go index aa7c6391..2b0b8947 100644 --- a/pkg/consumoor/router/engine.go +++ b/pkg/consumoor/router/engine.go @@ -86,12 +86,32 @@ func (r *Engine) Route(event *xatu.DecoratedEvent) Outcome { eventName := event.GetEvent().GetName() - // Look up routes for this event. Unknown event types are NAK'd - // (StatusErrored) so Kafka does not advance the offset. This - // prevents silent data loss when new event types appear before - // a matching route is deployed. + // Look up routes for this event. Intentionally unsupported events + // are dropped (status delivered, no rows). Unknown/unexpected event + // types are NAK'd (StatusErrored) so Kafka does not advance offsets, + // preventing silent data loss when new event types appear before a + // matching route is deployed. routesForEvent, ok := r.routesByEvent[eventName] if !ok { + if reason, intentionallyUnsupported := route.UnsupportedReason(eventName); intentionallyUnsupported { + if r.metrics != nil { + r.metrics.MessagesDropped().WithLabelValues(eventName.String(), "no_flattener").Inc() + } + + if ok, suppressed := r.logSampler.Allow("drop:" + eventName.String()); ok { + entry := r.log. + WithField("event_name", eventName.String()). + WithField("reason", reason) + if suppressed > 0 { + entry = entry.WithField("suppressed", suppressed) + } + + entry.Debug("No route registered for intentionally unsupported event — dropping") + } + + return Outcome{Status: StatusDelivered} + } + if r.metrics != nil { r.metrics.MessagesDropped().WithLabelValues(eventName.String(), "no_route_nack").Inc() } diff --git a/pkg/consumoor/router/engine_test.go b/pkg/consumoor/router/engine_test.go index cf63678e..c118f40e 100644 --- a/pkg/consumoor/router/engine_test.go +++ b/pkg/consumoor/router/engine_test.go @@ -67,6 +67,48 @@ func TestNewRouterSkipsDisabledEvents(t *testing.T) { require.Equal(t, "table_b", disconnectedRoutes[0].TableName()) } +func TestRouteIntentionallyUnsupportedEventIsDropped(t *testing.T) { + routes := []route.Route{ + filterTestRoute{ + table: route.TableName("table_head"), + events: []xatu.Event_Name{xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD_V2}, + }, + } + + router := New(logrus.New(), routes, nil, newTestMetrics()) + + outcome := router.Route(&xatu.DecoratedEvent{ + Event: &xatu.Event{ + Id: "e1", + Name: xatu.Event_BEACON_API_ETH_V1_DEBUG_FORK_CHOICE_V2, + }, + }) + + require.Equal(t, StatusDelivered, outcome.Status) + require.Empty(t, outcome.Results) +} + +func TestRouteUnknownEventIsNAKed(t *testing.T) { + routes := []route.Route{ + filterTestRoute{ + table: route.TableName("table_head"), + events: []xatu.Event_Name{xatu.Event_BEACON_API_ETH_V1_EVENTS_HEAD_V2}, + }, + } + + router := New(logrus.New(), routes, nil, newTestMetrics()) + + outcome := router.Route(&xatu.DecoratedEvent{ + Event: &xatu.Event{ + Id: "e1", + Name: xatu.Event_Name(999_999), + }, + }) + + require.Equal(t, StatusErrored, outcome.Status) + require.Empty(t, outcome.Results) +} + func newTestMetrics() *telemetry.Metrics { ns := fmt.Sprintf("xatu_consumoor_router_test_%d", atomic.AddUint64(&testMetricNSCounter, 1)) diff --git a/pkg/consumoor/source/benthos_test.go b/pkg/consumoor/source/benthos_test.go index 02fb8a08..6328759f 100644 --- a/pkg/consumoor/source/benthos_test.go +++ b/pkg/consumoor/source/benthos_test.go @@ -953,10 +953,9 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { assert.Equal(t, rejectReasonWritePermanent, rejectSink.records[0].Reason) }) - t.Run("unrouted_event_type_returns_batch_error", func(t *testing.T) { - // Event type has no registered route. The router returns - // StatusErrored so the message is NAK'd and Kafka does - // not advance the offset — preventing silent data loss. + t.Run("intentionally_unsupported_event_type_silently_commits", func(t *testing.T) { + // BLOCK is intentionally unsupported and should be dropped + // (ACK'd) without any ClickHouse writes. writer := &testWriter{} output := &xatuClickHouseOutput{ log: logrus.New(), @@ -973,11 +972,36 @@ func TestWriteBatchAtLeastOnceDelivery(t *testing.T) { newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_BEACON_API_ETH_V1_EVENTS_BLOCK), "t", 0, 1), } + err := output.WriteBatch(context.Background(), msgs) + require.NoError(t, err, + "intentionally unsupported event types should be dropped and ACK'd") + assert.Equal(t, 0, writer.flushCalls, + "no flush should occur for unsupported events") + }) + + t.Run("unknown_event_type_returns_batch_error", func(t *testing.T) { + // Unknown event enum values are not intentionally unsupported; + // they should NAK so offsets do not advance silently. + writer := &testWriter{} + output := &xatuClickHouseOutput{ + log: logrus.New(), + encoding: "json", + router: newRouter(t, []route.Route{headRoute}), + writer: writer, + metrics: newTestMetrics(), + logSampler: telemetry.NewLogSampler(time.Minute), + rejectSink: &testRejectSink{}, + } + + msgs := service.MessageBatch{ + newKafkaMessage(mustEventJSON(t, "e1", xatu.Event_Name(999_999)), "t", 0, 1), + } + err := output.WriteBatch(context.Background(), msgs) require.Error(t, err, - "unrouted event types must NAK to prevent silent data loss") + "unknown event types must NAK to avoid silent data loss") assert.Equal(t, 0, writer.flushCalls, - "no flush should occur for unrouted events") + "no flush should occur for unknown events") }) t.Run("nil_event_rejected_with_working_dlq_commits", func(t *testing.T) { From 975d7372cd8a03e7c64458679b070287b6d490ac Mon Sep 17 00:00:00 2001 From: Andrew Davis <1709934+Savid@users.noreply.github.com> Date: Wed, 1 Apr 2026 10:09:40 +1000 Subject: [PATCH 21/27] yeah (#804) --- .../migrations/clickhouse/001_init.down.sql | 0 deploy/migrations/clickhouse/001_init.up.sql | 5078 +++++++++++++++++ .../clickhouse/001_initialize_schema.down.sql | 22 - .../clickhouse/001_initialize_schema.up.sql | 467 -- .../002_attesting_validator.down.sql | 7 - .../clickhouse/002_attesting_validator.up.sql | 7 - .../003_remove_projections.down.sql | 47 - .../clickhouse/003_remove_projections.up.sql | 47 - deploy/migrations/clickhouse/004_dbt.down.sql | 1 - deploy/migrations/clickhouse/004_dbt.up.sql | 1 - .../clickhouse/005_beacon_committee.down.sql | 2 - .../clickhouse/005_beacon_committee.up.sql | 36 - .../clickhouse/006_comments.down.sql | 669 --- .../migrations/clickhouse/006_comments.up.sql | 669 --- deploy/migrations/clickhouse/007_tmp.down.sql | 1 - deploy/migrations/clickhouse/007_tmp.up.sql | 1 - .../008_block_transactions_size.down.sql | 7 - .../008_block_transactions_size.up.sql | 7 - .../009_beacon_block_v2_update.down.sql | 21 - .../009_beacon_block_v2_update.up.sql | 31 - .../010_validator_attestation_data.down.sql | 2 - .../010_validator_attestation_data.up.sql | 86 - .../011_block_transactions_size.down.sql | 9 - .../011_block_transactions_size.up.sql | 15 - .../migrations/clickhouse/012_cannon.down.sql | 12 - .../migrations/clickhouse/012_cannon.up.sql | 525 -- .../clickhouse/013_mempool_txn_type.down.sql | 11 - .../clickhouse/013_mempool_txn_type.up.sql | 17 - .../clickhouse/014_cannon_rename.down.sql | 22 - .../clickhouse/014_cannon_rename.up.sql | 728 --- .../clickhouse/015_cannon_replacing.down.sql | 721 --- .../clickhouse/015_cannon_replacing.up.sql | 750 --- .../clickhouse/016_blob_sidecar.down.sql | 2 - .../clickhouse/016_blob_sidecar.up.sql | 74 - .../clickhouse/017_cannon_blockprint.down.sql | 2 - .../clickhouse/017_cannon_blockprint.up.sql | 87 - .../018_cannon_blob_sidecar.down.sql | 2 - .../clickhouse/018_cannon_blob_sidecar.up.sql | 83 - .../019_blob_sidecar_versioned_hash.down.sql | 5 - .../019_blob_sidecar_versioned_hash.up.sql | 11 - .../clickhouse/020_imports.down.sql | 6 - .../migrations/clickhouse/020_imports.up.sql | 137 - .../clickhouse/021_fix_beacon_block.down.sql | 7 - .../clickhouse/021_fix_beacon_block.up.sql | 7 - .../022_beacon_p2p_attestation.down.sql | 2 - .../022_beacon_p2p_attestation.up.sql | 131 - .../023_cannon_proposer_duty.down.sql | 2 - .../023_cannon_proposer_duty.up.sql | 73 - ...024_cannon_elaborated_attestation.down.sql | 2 - .../024_cannon_elaborated_attestation.up.sql | 96 - ...5_rm_beacon_attestation_signature.down.sql | 11 - ...025_rm_beacon_attestation_signature.up.sql | 11 - .../026_dbt_model_metadata.down.sql | 2 - .../clickhouse/026_dbt_model_metadata.up.sql | 17 - .../clickhouse/027_blob_txn.down.sql | 169 - .../migrations/clickhouse/027_blob_txn.up.sql | 199 - .../clickhouse/028_comments.down.sql | 75 - .../migrations/clickhouse/028_comments.up.sql | 75 - ...ix_replacing_tables_sharding_keys.down.sql | 59 - ..._fix_replacing_tables_sharding_keys.up.sql | 59 - .../031_v1_proposer_duties.down.sql | 2 - .../clickhouse/031_v1_proposer_duties.up.sql | 73 - .../clickhouse/032_libp2p_trace.down.sql | 47 - .../clickhouse/032_libp2p_trace.up.sql | 904 --- .../033_libp2p_trace_metadata_status.down.sql | 5 - .../033_libp2p_trace_metadata_status.up.sql | 142 - ...034_libp2p_gossipsub_beacon_block.down.sql | 2 - .../034_libp2p_gossipsub_beacon_block.up.sql | 84 - ...bp2p_gossipsub_beacon_attestation.down.sql | 2 - ...libp2p_gossipsub_beacon_attestation.up.sql | 102 - .../036_replacing_merge_trees.down.sql | 1104 ---- .../036_replacing_merge_trees.up.sql | 1104 ---- ...037_libp2p_gossipsub_blob_sidecar.down.sql | 2 - .../037_libp2p_gossipsub_blob_sidecar.up.sql | 88 - ...ssipsub_blob_sidecar_block_fields.down.sql | 11 - ...gossipsub_blob_sidecar_block_fields.up.sql | 23 - .../039_canonical_beacon_validators.down.sql | 2 - .../039_canonical_beacon_validators.up.sql | 85 - ...canonical_beacon_validators_split.down.sql | 94 - ...0_canonical_beacon_validators_split.up.sql | 226 - .../041_remove_unique_field.down.sql | 4594 --------------- .../clickhouse/041_remove_unique_field.up.sql | 4780 ---------------- ...042_delete_beacon_p2p_attestation.down.sql | 95 - .../042_delete_beacon_p2p_attestation.up.sql | 2 - ...anonical_beacon_validators_tweaks.down.sql | 56 - ..._canonical_beacon_validators_tweaks.up.sql | 68 - ...4_events_voluntary_exit_wallclock.down.sql | 108 - ...044_events_voluntary_exit_wallclock.up.sql | 124 - .../045_canonical_beacon_committee.down.sql | 2 - .../045_canonical_beacon_committee.up.sql | 53 - .../046_remove_canonical_event_time.down.sql | 294 - .../046_remove_canonical_event_time.up.sql | 264 - .../047_block_execution_payloads.down.sql | 27 - .../047_block_execution_payloads.up.sql | 27 - ..._canonical_beacon_block_execution.down.sql | 13 - ...le_canonical_beacon_block_execution.up.sql | 13 - .../049_canonical_execution.down.sql | 34 - .../clickhouse/049_canonical_execution.up.sql | 634 -- .../050_ethseer_validator_entity.down.sql | 2 - .../050_ethseer_validator_entity.up.sql | 21 - .../051_mev_relay_block_bids.down.sql | 2 - .../051_mev_relay_block_bids.up.sql | 68 - ..._relay_proposer_payload_delivered.down.sql | 2 - ...ev_relay_proposer_payload_delivered.up.sql | 61 - ...3_add_bid_trace_relay_name_unique.down.sql | 5 - ...053_add_bid_trace_relay_name_unique.up.sql | 72 - .../clickhouse/054_validator_block.down.sql | 2 - .../clickhouse/054_validator_block.up.sql | 70 - ...mev_relay_validator_registrations.down.sql | 2 - ...5_mev_relay_validator_registrations.up.sql | 56 - ...p_ordered_validator_registrations.down.sql | 59 - ...amp_ordered_validator_registrations.up.sql | 59 - .../clickhouse/057_block_gossip.down.sql | 2 - .../clickhouse/057_block_gossip.up.sql | 55 - .../clickhouse/058_libp2p_drop_rpc.down.sql | 2 - .../clickhouse/058_libp2p_drop_rpc.up.sql | 43 - .../059_libp2p_trace_leave.down.sql | 2 - .../clickhouse/059_libp2p_trace_leave.up.sql | 53 - .../060_libp2p_trace_graft.down.sql | 2 - .../clickhouse/060_libp2p_trace_graft.up.sql | 53 - .../061_libp2p_trace_prune.down.sql | 2 - .../clickhouse/061_libp2p_trace_prune.up.sql | 53 - .../062_libp2p_trace_deliver_message.down.sql | 2 - .../062_libp2p_trace_deliver_message.up.sql | 61 - .../063_libp2p_trace_reject_message.down.sql | 2 - .../063_libp2p_trace_reject_message.up.sql | 62 - .../064_libp2p_trace_publish_message.down.sql | 2 - .../064_libp2p_trace_publish_message.up.sql | 53 - ...65_libp2p_trace_duplicate_message.down.sql | 2 - .../065_libp2p_trace_duplicate_message.up.sql | 61 - ...libp2p_rpc_meta_control_idontwant.down.sql | 2 - ...6_libp2p_rpc_meta_control_idontwant.up.sql | 33 - .../067_transactions_structlog.down.sql | 5 - .../067_transactions_structlog.up.sql | 73 - ...68_nullable_libp2p_rpc_meta_prune.down.sql | 6 - .../068_nullable_libp2p_rpc_meta_prune.up.sql | 6 - ...69_transactions_structlog_call_to.down.sql | 5 - .../069_transactions_structlog_call_to.up.sql | 5 - ...p2p_gossipsub_aggregate_and_proof.down.sql | 2 - ...ibp2p_gossipsub_aggregate_and_proof.up.sql | 77 - .../071_data_column_sidecar.down.sql | 3 - .../clickhouse/071_data_column_sidecar.up.sql | 59 - .../072_node_record_execution.down.sql | 2 - .../072_node_record_execution.up.sql | 71 - .../073_node_record_consensus.down.sql | 2 - .../073_node_record_consensus.up.sql | 76 - ...p2p_gossipsub_data_column_sidecar.down.sql | 2 - ...ibp2p_gossipsub_data_column_sidecar.up.sql | 68 - .../075_libp2p_synthetic_heartbeat.down.sql | 2 - .../075_libp2p_synthetic_heartbeat.up.sql | 113 - .../clickhouse/076_block_root_libp2p.down.sql | 11 - .../clickhouse/076_block_root_libp2p.up.sql | 12 - ...umn_sidecar_kzg_commitments_count.down.sql | 20 - ...olumn_sidecar_kzg_commitments_count.up.sql | 20 - .../078_add_peerdas_rpc_columns.down.sql | 48 - .../078_add_peerdas_rpc_columns.up.sql | 57 - .../clickhouse/079_custody_probe.down.sql | 2 - .../clickhouse/079_custody_probe.up.sql | 80 - .../clickhouse/080_destroy_dbt.down.sql | 1 - .../clickhouse/080_destroy_dbt.up.sql | 1 - .../081_execution_state_size.down.sql | 2 - .../081_execution_state_size.up.sql | 65 - .../082_execution_transaction.down.sql | 2 - .../082_execution_transaction.up.sql | 51 - .../clickhouse/083_blob_submitter.down.sql | 2 - .../clickhouse/083_blob_submitter.up.sql | 25 - ..._consensus_engine_api_new_payload.down.sql | 2 - ...84_consensus_engine_api_new_payload.up.sql | 69 - ...api_new_payload_execution_version.down.sql | 31 - ...e_api_new_payload_execution_version.up.sql | 32 - ...86_consensus_engine_api_get_blobs.down.sql | 2 - .../086_consensus_engine_api_get_blobs.up.sql | 70 - ..._proposer_payload_delivered_value.down.sql | 5 - ...ay_proposer_payload_delivered_value.up.sql | 5 - ...88_transactions_structlog_call_to.down.sql | 5 - .../088_transactions_structlog_call_to.up.sql | 5 - .../clickhouse/089_table_comments.down.sql | 629 -- .../clickhouse/089_table_comments.up.sql | 643 --- .../090_execution_engine_events.down.sql | 4 - .../090_execution_engine_events.up.sql | 134 - ...nonical_execution_block_gas_limit.down.sql | 5 - ...canonical_execution_block_gas_limit.up.sql | 5 - ...engine_get_blobs_returned_indexes.down.sql | 5 - ...2_engine_get_blobs_returned_indexes.up.sql | 9 - .../clickhouse/093_beacon_blob.down.sql | 2 - .../clickhouse/093_beacon_blob.up.sql | 76 - .../094_structlog_call_frames.down.sql | 45 - .../094_structlog_call_frames.up.sql | 91 - ...5_canonical_beacon_sync_committee.down.sql | 2 - ...095_canonical_beacon_sync_committee.up.sql | 51 - ...nical_beacon_block_sync_aggregate.down.sql | 2 - ...nonical_beacon_block_sync_aggregate.up.sql | 59 - .../097_observoor_initialize.down.sql | 89 - .../097_observoor_initialize.up.sql | 1119 ---- ...098_observoor_float32_sum_min_max.down.sql | 293 - .../098_observoor_float32_sum_min_max.up.sql | 295 - .../099_observoor_cpu_utilization.down.sql | 2 - .../099_observoor_cpu_utilization.up.sql | 36 - .../100_observoor_port_labels.down.sql | 132 - .../100_observoor_port_labels.up.sql | 133 - .../101_observoor_sampling_metadata.down.sql | 280 - .../101_observoor_sampling_metadata.up.sql | 281 - .../102_execution_block_metrics.down.sql | 2 - .../102_execution_block_metrics.up.sql | 89 - .../103_observoor_process_snapshots.down.sql | 14 - .../103_observoor_process_snapshots.up.sql | 198 - ...tructlog_agg_resource_gas_columns.down.sql | 15 - ..._structlog_agg_resource_gas_columns.up.sql | 19 - .../clickhouse/105_libp2p_identify.down.sql | 2 - .../clickhouse/105_libp2p_identify.up.sql | 65 - .../beacon_api_eth_v1_beacon_blob.gen.go | 29 +- .../beacon/beacon_api_eth_v1_beacon_blob.go | 2 + .../beacon_api_eth_v1_beacon_committee.gen.go | 25 +- ...eacon_api_eth_v1_events_attestation.gen.go | 29 +- .../beacon_api_eth_v1_events_attestation.go | 2 + ...acon_api_eth_v1_events_blob_sidecar.gen.go | 25 +- .../beacon_api_eth_v1_events_block.gen.go | 25 +- ...acon_api_eth_v1_events_block_gossip.gen.go | 25 +- ...eacon_api_eth_v1_events_chain_reorg.gen.go | 25 +- ...th_v1_events_contribution_and_proof.gen.go | 25 +- ...i_eth_v1_events_data_column_sidecar.gen.go | 38 +- ...n_api_eth_v1_events_data_column_sidecar.go | 8 + ..._eth_v1_events_finalized_checkpoint.gen.go | 25 +- .../beacon_api_eth_v1_events_head.gen.go | 25 +- ...on_api_eth_v1_events_voluntary_exit.gen.go | 25 +- .../beacon_api_eth_v1_proposer_duty.gen.go | 25 +- ...i_eth_v1_validator_attestation_data.gen.go | 25 +- .../beacon_api_eth_v2_beacon_block.gen.go | 59 +- .../beacon/beacon_api_eth_v2_beacon_block.go | 38 +- .../beacon_api_eth_v3_validator_block.gen.go | 25 +- .../canonical_beacon_blob_sidecar.gen.go | 205 +- .../canonical/canonical_beacon_block.gen.go | 161 +- ...ical_beacon_block_attester_slashing.gen.go | 233 +- ...cal_beacon_block_attester_slashing_test.go | 2 +- ...eacon_block_bls_to_execution_change.gen.go | 191 +- ...acon_block_bls_to_execution_change_test.go | 2 +- .../canonical_beacon_block_deposit.gen.go | 195 +- .../canonical_beacon_block_deposit_test.go | 2 +- ..._beacon_block_execution_transaction.gen.go | 237 +- ...ical_beacon_block_proposer_slashing.gen.go | 217 +- ...cal_beacon_block_proposer_slashing_test.go | 2 +- ...nonical_beacon_block_sync_aggregate.gen.go | 201 +- ...onical_beacon_block_sync_aggregate_test.go | 2 +- ...nonical_beacon_block_voluntary_exit.gen.go | 187 +- ...onical_beacon_block_voluntary_exit_test.go | 2 +- .../canonical_beacon_block_withdrawal.gen.go | 191 +- .../canonical_beacon_committee.gen.go | 181 +- ...nical_beacon_elaborated_attestation.gen.go | 215 +- ...ical_beacon_elaborated_attestation_test.go | 2 +- .../canonical_beacon_proposer_duty.gen.go | 182 +- .../canonical_beacon_sync_committee.gen.go | 181 +- .../canonical_beacon_sync_committee_test.go | 2 +- .../canonical_beacon_validators.gen.go | 201 +- ...canonical_beacon_validators_pubkeys.gen.go | 182 +- .../canonical_beacon_validators_pubkeys.go | 1 - ...n_validators_withdrawal_credentials.gen.go | 182 +- ...eacon_validators_withdrawal_credentials.go | 1 - .../consensus_engine_api_get_blobs.gen.go | 14 +- .../consensus_engine_api_new_payload.gen.go | 14 +- .../execution/execution_block_metrics.gen.go | 25 +- .../execution_engine_get_blobs.gen.go | 16 +- .../execution/execution_engine_get_blobs.go | 2 +- .../execution_engine_new_payload.gen.go | 16 +- .../execution/execution_engine_new_payload.go | 2 +- .../execution/execution_state_size.gen.go | 25 +- .../execution/mempool_transaction.gen.go | 25 +- .../route/libp2p/libp2p_add_peer.gen.go | 14 +- .../route/libp2p/libp2p_connected.gen.go | 25 +- .../route/libp2p/libp2p_connected.go | 6 +- .../libp2p/libp2p_deliver_message.gen.go | 14 +- .../route/libp2p/libp2p_disconnected.gen.go | 25 +- .../route/libp2p/libp2p_disconnected.go | 6 +- .../route/libp2p/libp2p_drop_rpc.gen.go | 14 +- .../libp2p/libp2p_duplicate_message.gen.go | 14 +- ...ibp2p_gossipsub_aggregate_and_proof.gen.go | 18 +- .../libp2p_gossipsub_aggregate_and_proof.go | 8 - ...libp2p_gossipsub_beacon_attestation.gen.go | 18 +- .../libp2p_gossipsub_beacon_attestation.go | 8 - .../libp2p_gossipsub_beacon_block.gen.go | 18 +- .../libp2p/libp2p_gossipsub_beacon_block.go | 8 - .../libp2p_gossipsub_blob_sidecar.gen.go | 18 +- .../libp2p/libp2p_gossipsub_blob_sidecar.go | 8 - ...ibp2p_gossipsub_data_column_sidecar.gen.go | 18 +- .../libp2p_gossipsub_data_column_sidecar.go | 8 - .../route/libp2p/libp2p_graft.gen.go | 14 +- .../libp2p/libp2p_handle_metadata.gen.go | 25 +- .../route/libp2p/libp2p_handle_metadata.go | 7 +- .../route/libp2p/libp2p_handle_status.gen.go | 25 +- .../route/libp2p/libp2p_handle_status.go | 7 +- .../route/libp2p/libp2p_identify.gen.go | 14 +- pkg/consumoor/route/libp2p/libp2p_join.gen.go | 14 +- .../route/libp2p/libp2p_leave.gen.go | 14 +- pkg/consumoor/route/libp2p/libp2p_peer.gen.go | 8 +- .../route/libp2p/libp2p_prune.gen.go | 14 +- .../libp2p/libp2p_publish_message.gen.go | 14 +- .../route/libp2p/libp2p_recv_rpc.gen.go | 14 +- .../route/libp2p/libp2p_reject_message.gen.go | 14 +- .../route/libp2p/libp2p_remove_peer.gen.go | 14 +- ...ibp2p_rpc_data_column_custody_probe.gen.go | 25 +- .../libp2p_rpc_meta_control_graft.gen.go | 14 +- .../libp2p_rpc_meta_control_idontwant.gen.go | 14 +- .../libp2p_rpc_meta_control_ihave.gen.go | 14 +- .../libp2p_rpc_meta_control_iwant.gen.go | 14 +- .../libp2p_rpc_meta_control_prune.gen.go | 14 +- .../libp2p/libp2p_rpc_meta_message.gen.go | 14 +- .../libp2p_rpc_meta_subscription.gen.go | 14 +- .../route/libp2p/libp2p_send_rpc.gen.go | 14 +- .../libp2p/libp2p_synthetic_heartbeat.gen.go | 14 +- .../route/mev/mev_relay_bid_trace.gen.go | 19 +- ...ev_relay_proposer_payload_delivered.gen.go | 19 +- .../mev_relay_validator_registration.gen.go | 19 +- .../route/node/node_record_consensus.gen.go | 36 +- .../route/node/node_record_consensus.go | 8 +- .../route/node/node_record_execution.gen.go | 25 +- 314 files changed, 5573 insertions(+), 32925 deletions(-) create mode 100644 deploy/migrations/clickhouse/001_init.down.sql create mode 100644 deploy/migrations/clickhouse/001_init.up.sql delete mode 100644 deploy/migrations/clickhouse/001_initialize_schema.down.sql delete mode 100644 deploy/migrations/clickhouse/001_initialize_schema.up.sql delete mode 100644 deploy/migrations/clickhouse/002_attesting_validator.down.sql delete mode 100644 deploy/migrations/clickhouse/002_attesting_validator.up.sql delete mode 100644 deploy/migrations/clickhouse/003_remove_projections.down.sql delete mode 100644 deploy/migrations/clickhouse/003_remove_projections.up.sql delete mode 100644 deploy/migrations/clickhouse/004_dbt.down.sql delete mode 100644 deploy/migrations/clickhouse/004_dbt.up.sql delete mode 100644 deploy/migrations/clickhouse/005_beacon_committee.down.sql delete mode 100644 deploy/migrations/clickhouse/005_beacon_committee.up.sql delete mode 100644 deploy/migrations/clickhouse/006_comments.down.sql delete mode 100644 deploy/migrations/clickhouse/006_comments.up.sql delete mode 100644 deploy/migrations/clickhouse/007_tmp.down.sql delete mode 100644 deploy/migrations/clickhouse/007_tmp.up.sql delete mode 100644 deploy/migrations/clickhouse/008_block_transactions_size.down.sql delete mode 100644 deploy/migrations/clickhouse/008_block_transactions_size.up.sql delete mode 100644 deploy/migrations/clickhouse/009_beacon_block_v2_update.down.sql delete mode 100644 deploy/migrations/clickhouse/009_beacon_block_v2_update.up.sql delete mode 100644 deploy/migrations/clickhouse/010_validator_attestation_data.down.sql delete mode 100644 deploy/migrations/clickhouse/010_validator_attestation_data.up.sql delete mode 100644 deploy/migrations/clickhouse/011_block_transactions_size.down.sql delete mode 100644 deploy/migrations/clickhouse/011_block_transactions_size.up.sql delete mode 100644 deploy/migrations/clickhouse/012_cannon.down.sql delete mode 100644 deploy/migrations/clickhouse/012_cannon.up.sql delete mode 100644 deploy/migrations/clickhouse/013_mempool_txn_type.down.sql delete mode 100644 deploy/migrations/clickhouse/013_mempool_txn_type.up.sql delete mode 100644 deploy/migrations/clickhouse/014_cannon_rename.down.sql delete mode 100644 deploy/migrations/clickhouse/014_cannon_rename.up.sql delete mode 100644 deploy/migrations/clickhouse/015_cannon_replacing.down.sql delete mode 100644 deploy/migrations/clickhouse/015_cannon_replacing.up.sql delete mode 100644 deploy/migrations/clickhouse/016_blob_sidecar.down.sql delete mode 100644 deploy/migrations/clickhouse/016_blob_sidecar.up.sql delete mode 100644 deploy/migrations/clickhouse/017_cannon_blockprint.down.sql delete mode 100644 deploy/migrations/clickhouse/017_cannon_blockprint.up.sql delete mode 100644 deploy/migrations/clickhouse/018_cannon_blob_sidecar.down.sql delete mode 100644 deploy/migrations/clickhouse/018_cannon_blob_sidecar.up.sql delete mode 100644 deploy/migrations/clickhouse/019_blob_sidecar_versioned_hash.down.sql delete mode 100644 deploy/migrations/clickhouse/019_blob_sidecar_versioned_hash.up.sql delete mode 100644 deploy/migrations/clickhouse/020_imports.down.sql delete mode 100644 deploy/migrations/clickhouse/020_imports.up.sql delete mode 100644 deploy/migrations/clickhouse/021_fix_beacon_block.down.sql delete mode 100644 deploy/migrations/clickhouse/021_fix_beacon_block.up.sql delete mode 100644 deploy/migrations/clickhouse/022_beacon_p2p_attestation.down.sql delete mode 100644 deploy/migrations/clickhouse/022_beacon_p2p_attestation.up.sql delete mode 100644 deploy/migrations/clickhouse/023_cannon_proposer_duty.down.sql delete mode 100644 deploy/migrations/clickhouse/023_cannon_proposer_duty.up.sql delete mode 100644 deploy/migrations/clickhouse/024_cannon_elaborated_attestation.down.sql delete mode 100644 deploy/migrations/clickhouse/024_cannon_elaborated_attestation.up.sql delete mode 100644 deploy/migrations/clickhouse/025_rm_beacon_attestation_signature.down.sql delete mode 100644 deploy/migrations/clickhouse/025_rm_beacon_attestation_signature.up.sql delete mode 100644 deploy/migrations/clickhouse/026_dbt_model_metadata.down.sql delete mode 100644 deploy/migrations/clickhouse/026_dbt_model_metadata.up.sql delete mode 100644 deploy/migrations/clickhouse/027_blob_txn.down.sql delete mode 100644 deploy/migrations/clickhouse/027_blob_txn.up.sql delete mode 100644 deploy/migrations/clickhouse/028_comments.down.sql delete mode 100644 deploy/migrations/clickhouse/028_comments.up.sql delete mode 100644 deploy/migrations/clickhouse/030_fix_replacing_tables_sharding_keys.down.sql delete mode 100644 deploy/migrations/clickhouse/030_fix_replacing_tables_sharding_keys.up.sql delete mode 100644 deploy/migrations/clickhouse/031_v1_proposer_duties.down.sql delete mode 100644 deploy/migrations/clickhouse/031_v1_proposer_duties.up.sql delete mode 100644 deploy/migrations/clickhouse/032_libp2p_trace.down.sql delete mode 100644 deploy/migrations/clickhouse/032_libp2p_trace.up.sql delete mode 100644 deploy/migrations/clickhouse/033_libp2p_trace_metadata_status.down.sql delete mode 100644 deploy/migrations/clickhouse/033_libp2p_trace_metadata_status.up.sql delete mode 100644 deploy/migrations/clickhouse/034_libp2p_gossipsub_beacon_block.down.sql delete mode 100644 deploy/migrations/clickhouse/034_libp2p_gossipsub_beacon_block.up.sql delete mode 100644 deploy/migrations/clickhouse/035_libp2p_gossipsub_beacon_attestation.down.sql delete mode 100644 deploy/migrations/clickhouse/035_libp2p_gossipsub_beacon_attestation.up.sql delete mode 100644 deploy/migrations/clickhouse/036_replacing_merge_trees.down.sql delete mode 100644 deploy/migrations/clickhouse/036_replacing_merge_trees.up.sql delete mode 100644 deploy/migrations/clickhouse/037_libp2p_gossipsub_blob_sidecar.down.sql delete mode 100644 deploy/migrations/clickhouse/037_libp2p_gossipsub_blob_sidecar.up.sql delete mode 100644 deploy/migrations/clickhouse/038_libp2p_gossipsub_blob_sidecar_block_fields.down.sql delete mode 100644 deploy/migrations/clickhouse/038_libp2p_gossipsub_blob_sidecar_block_fields.up.sql delete mode 100644 deploy/migrations/clickhouse/039_canonical_beacon_validators.down.sql delete mode 100644 deploy/migrations/clickhouse/039_canonical_beacon_validators.up.sql delete mode 100644 deploy/migrations/clickhouse/040_canonical_beacon_validators_split.down.sql delete mode 100644 deploy/migrations/clickhouse/040_canonical_beacon_validators_split.up.sql delete mode 100644 deploy/migrations/clickhouse/041_remove_unique_field.down.sql delete mode 100644 deploy/migrations/clickhouse/041_remove_unique_field.up.sql delete mode 100644 deploy/migrations/clickhouse/042_delete_beacon_p2p_attestation.down.sql delete mode 100644 deploy/migrations/clickhouse/042_delete_beacon_p2p_attestation.up.sql delete mode 100644 deploy/migrations/clickhouse/043_canonical_beacon_validators_tweaks.down.sql delete mode 100644 deploy/migrations/clickhouse/043_canonical_beacon_validators_tweaks.up.sql delete mode 100644 deploy/migrations/clickhouse/044_events_voluntary_exit_wallclock.down.sql delete mode 100644 deploy/migrations/clickhouse/044_events_voluntary_exit_wallclock.up.sql delete mode 100644 deploy/migrations/clickhouse/045_canonical_beacon_committee.down.sql delete mode 100644 deploy/migrations/clickhouse/045_canonical_beacon_committee.up.sql delete mode 100644 deploy/migrations/clickhouse/046_remove_canonical_event_time.down.sql delete mode 100644 deploy/migrations/clickhouse/046_remove_canonical_event_time.up.sql delete mode 100644 deploy/migrations/clickhouse/047_block_execution_payloads.down.sql delete mode 100644 deploy/migrations/clickhouse/047_block_execution_payloads.up.sql delete mode 100644 deploy/migrations/clickhouse/048_nullable_canonical_beacon_block_execution.down.sql delete mode 100644 deploy/migrations/clickhouse/048_nullable_canonical_beacon_block_execution.up.sql delete mode 100644 deploy/migrations/clickhouse/049_canonical_execution.down.sql delete mode 100644 deploy/migrations/clickhouse/049_canonical_execution.up.sql delete mode 100644 deploy/migrations/clickhouse/050_ethseer_validator_entity.down.sql delete mode 100644 deploy/migrations/clickhouse/050_ethseer_validator_entity.up.sql delete mode 100644 deploy/migrations/clickhouse/051_mev_relay_block_bids.down.sql delete mode 100644 deploy/migrations/clickhouse/051_mev_relay_block_bids.up.sql delete mode 100644 deploy/migrations/clickhouse/052_mev_relay_proposer_payload_delivered.down.sql delete mode 100644 deploy/migrations/clickhouse/052_mev_relay_proposer_payload_delivered.up.sql delete mode 100644 deploy/migrations/clickhouse/053_add_bid_trace_relay_name_unique.down.sql delete mode 100644 deploy/migrations/clickhouse/053_add_bid_trace_relay_name_unique.up.sql delete mode 100644 deploy/migrations/clickhouse/054_validator_block.down.sql delete mode 100644 deploy/migrations/clickhouse/054_validator_block.up.sql delete mode 100644 deploy/migrations/clickhouse/055_mev_relay_validator_registrations.down.sql delete mode 100644 deploy/migrations/clickhouse/055_mev_relay_validator_registrations.up.sql delete mode 100644 deploy/migrations/clickhouse/056_event_timestamp_ordered_validator_registrations.down.sql delete mode 100644 deploy/migrations/clickhouse/056_event_timestamp_ordered_validator_registrations.up.sql delete mode 100644 deploy/migrations/clickhouse/057_block_gossip.down.sql delete mode 100644 deploy/migrations/clickhouse/057_block_gossip.up.sql delete mode 100644 deploy/migrations/clickhouse/058_libp2p_drop_rpc.down.sql delete mode 100644 deploy/migrations/clickhouse/058_libp2p_drop_rpc.up.sql delete mode 100644 deploy/migrations/clickhouse/059_libp2p_trace_leave.down.sql delete mode 100644 deploy/migrations/clickhouse/059_libp2p_trace_leave.up.sql delete mode 100644 deploy/migrations/clickhouse/060_libp2p_trace_graft.down.sql delete mode 100644 deploy/migrations/clickhouse/060_libp2p_trace_graft.up.sql delete mode 100644 deploy/migrations/clickhouse/061_libp2p_trace_prune.down.sql delete mode 100644 deploy/migrations/clickhouse/061_libp2p_trace_prune.up.sql delete mode 100644 deploy/migrations/clickhouse/062_libp2p_trace_deliver_message.down.sql delete mode 100644 deploy/migrations/clickhouse/062_libp2p_trace_deliver_message.up.sql delete mode 100644 deploy/migrations/clickhouse/063_libp2p_trace_reject_message.down.sql delete mode 100644 deploy/migrations/clickhouse/063_libp2p_trace_reject_message.up.sql delete mode 100644 deploy/migrations/clickhouse/064_libp2p_trace_publish_message.down.sql delete mode 100644 deploy/migrations/clickhouse/064_libp2p_trace_publish_message.up.sql delete mode 100644 deploy/migrations/clickhouse/065_libp2p_trace_duplicate_message.down.sql delete mode 100644 deploy/migrations/clickhouse/065_libp2p_trace_duplicate_message.up.sql delete mode 100644 deploy/migrations/clickhouse/066_libp2p_rpc_meta_control_idontwant.down.sql delete mode 100644 deploy/migrations/clickhouse/066_libp2p_rpc_meta_control_idontwant.up.sql delete mode 100644 deploy/migrations/clickhouse/067_transactions_structlog.down.sql delete mode 100644 deploy/migrations/clickhouse/067_transactions_structlog.up.sql delete mode 100644 deploy/migrations/clickhouse/068_nullable_libp2p_rpc_meta_prune.down.sql delete mode 100644 deploy/migrations/clickhouse/068_nullable_libp2p_rpc_meta_prune.up.sql delete mode 100644 deploy/migrations/clickhouse/069_transactions_structlog_call_to.down.sql delete mode 100644 deploy/migrations/clickhouse/069_transactions_structlog_call_to.up.sql delete mode 100644 deploy/migrations/clickhouse/070_libp2p_gossipsub_aggregate_and_proof.down.sql delete mode 100644 deploy/migrations/clickhouse/070_libp2p_gossipsub_aggregate_and_proof.up.sql delete mode 100644 deploy/migrations/clickhouse/071_data_column_sidecar.down.sql delete mode 100644 deploy/migrations/clickhouse/071_data_column_sidecar.up.sql delete mode 100644 deploy/migrations/clickhouse/072_node_record_execution.down.sql delete mode 100644 deploy/migrations/clickhouse/072_node_record_execution.up.sql delete mode 100644 deploy/migrations/clickhouse/073_node_record_consensus.down.sql delete mode 100644 deploy/migrations/clickhouse/073_node_record_consensus.up.sql delete mode 100644 deploy/migrations/clickhouse/074_libp2p_gossipsub_data_column_sidecar.down.sql delete mode 100644 deploy/migrations/clickhouse/074_libp2p_gossipsub_data_column_sidecar.up.sql delete mode 100644 deploy/migrations/clickhouse/075_libp2p_synthetic_heartbeat.down.sql delete mode 100644 deploy/migrations/clickhouse/075_libp2p_synthetic_heartbeat.up.sql delete mode 100644 deploy/migrations/clickhouse/076_block_root_libp2p.down.sql delete mode 100644 deploy/migrations/clickhouse/076_block_root_libp2p.up.sql delete mode 100644 deploy/migrations/clickhouse/077_data_column_sidecar_kzg_commitments_count.down.sql delete mode 100644 deploy/migrations/clickhouse/077_data_column_sidecar_kzg_commitments_count.up.sql delete mode 100644 deploy/migrations/clickhouse/078_add_peerdas_rpc_columns.down.sql delete mode 100644 deploy/migrations/clickhouse/078_add_peerdas_rpc_columns.up.sql delete mode 100644 deploy/migrations/clickhouse/079_custody_probe.down.sql delete mode 100644 deploy/migrations/clickhouse/079_custody_probe.up.sql delete mode 100644 deploy/migrations/clickhouse/080_destroy_dbt.down.sql delete mode 100644 deploy/migrations/clickhouse/080_destroy_dbt.up.sql delete mode 100644 deploy/migrations/clickhouse/081_execution_state_size.down.sql delete mode 100644 deploy/migrations/clickhouse/081_execution_state_size.up.sql delete mode 100644 deploy/migrations/clickhouse/082_execution_transaction.down.sql delete mode 100644 deploy/migrations/clickhouse/082_execution_transaction.up.sql delete mode 100644 deploy/migrations/clickhouse/083_blob_submitter.down.sql delete mode 100644 deploy/migrations/clickhouse/083_blob_submitter.up.sql delete mode 100644 deploy/migrations/clickhouse/084_consensus_engine_api_new_payload.down.sql delete mode 100644 deploy/migrations/clickhouse/084_consensus_engine_api_new_payload.up.sql delete mode 100644 deploy/migrations/clickhouse/085_consensus_engine_api_new_payload_execution_version.down.sql delete mode 100644 deploy/migrations/clickhouse/085_consensus_engine_api_new_payload_execution_version.up.sql delete mode 100644 deploy/migrations/clickhouse/086_consensus_engine_api_get_blobs.down.sql delete mode 100644 deploy/migrations/clickhouse/086_consensus_engine_api_get_blobs.up.sql delete mode 100644 deploy/migrations/clickhouse/087_mev_relay_proposer_payload_delivered_value.down.sql delete mode 100644 deploy/migrations/clickhouse/087_mev_relay_proposer_payload_delivered_value.up.sql delete mode 100644 deploy/migrations/clickhouse/088_transactions_structlog_call_to.down.sql delete mode 100644 deploy/migrations/clickhouse/088_transactions_structlog_call_to.up.sql delete mode 100644 deploy/migrations/clickhouse/089_table_comments.down.sql delete mode 100644 deploy/migrations/clickhouse/089_table_comments.up.sql delete mode 100644 deploy/migrations/clickhouse/090_execution_engine_events.down.sql delete mode 100644 deploy/migrations/clickhouse/090_execution_engine_events.up.sql delete mode 100644 deploy/migrations/clickhouse/091_canonical_execution_block_gas_limit.down.sql delete mode 100644 deploy/migrations/clickhouse/091_canonical_execution_block_gas_limit.up.sql delete mode 100644 deploy/migrations/clickhouse/092_engine_get_blobs_returned_indexes.down.sql delete mode 100644 deploy/migrations/clickhouse/092_engine_get_blobs_returned_indexes.up.sql delete mode 100644 deploy/migrations/clickhouse/093_beacon_blob.down.sql delete mode 100644 deploy/migrations/clickhouse/093_beacon_blob.up.sql delete mode 100644 deploy/migrations/clickhouse/094_structlog_call_frames.down.sql delete mode 100644 deploy/migrations/clickhouse/094_structlog_call_frames.up.sql delete mode 100644 deploy/migrations/clickhouse/095_canonical_beacon_sync_committee.down.sql delete mode 100644 deploy/migrations/clickhouse/095_canonical_beacon_sync_committee.up.sql delete mode 100644 deploy/migrations/clickhouse/096_canonical_beacon_block_sync_aggregate.down.sql delete mode 100644 deploy/migrations/clickhouse/096_canonical_beacon_block_sync_aggregate.up.sql delete mode 100644 deploy/migrations/clickhouse/097_observoor_initialize.down.sql delete mode 100644 deploy/migrations/clickhouse/097_observoor_initialize.up.sql delete mode 100644 deploy/migrations/clickhouse/098_observoor_float32_sum_min_max.down.sql delete mode 100644 deploy/migrations/clickhouse/098_observoor_float32_sum_min_max.up.sql delete mode 100644 deploy/migrations/clickhouse/099_observoor_cpu_utilization.down.sql delete mode 100644 deploy/migrations/clickhouse/099_observoor_cpu_utilization.up.sql delete mode 100644 deploy/migrations/clickhouse/100_observoor_port_labels.down.sql delete mode 100644 deploy/migrations/clickhouse/100_observoor_port_labels.up.sql delete mode 100644 deploy/migrations/clickhouse/101_observoor_sampling_metadata.down.sql delete mode 100644 deploy/migrations/clickhouse/101_observoor_sampling_metadata.up.sql delete mode 100644 deploy/migrations/clickhouse/102_execution_block_metrics.down.sql delete mode 100644 deploy/migrations/clickhouse/102_execution_block_metrics.up.sql delete mode 100644 deploy/migrations/clickhouse/103_observoor_process_snapshots.down.sql delete mode 100644 deploy/migrations/clickhouse/103_observoor_process_snapshots.up.sql delete mode 100644 deploy/migrations/clickhouse/104_structlog_agg_resource_gas_columns.down.sql delete mode 100644 deploy/migrations/clickhouse/104_structlog_agg_resource_gas_columns.up.sql delete mode 100644 deploy/migrations/clickhouse/105_libp2p_identify.down.sql delete mode 100644 deploy/migrations/clickhouse/105_libp2p_identify.up.sql diff --git a/deploy/migrations/clickhouse/001_init.down.sql b/deploy/migrations/clickhouse/001_init.down.sql new file mode 100644 index 00000000..e69de29b diff --git a/deploy/migrations/clickhouse/001_init.up.sql b/deploy/migrations/clickhouse/001_init.up.sql new file mode 100644 index 00000000..4f8090f5 --- /dev/null +++ b/deploy/migrations/clickhouse/001_init.up.sql @@ -0,0 +1,5078 @@ +-- Migration 102: Schema V2 (GENERATED) +-- +-- Source: schemas + spec.yaml +-- This file is generated by generate_migration.py. + +CREATE DATABASE IF NOT EXISTS observoor ON CLUSTER '{cluster}'; + +CREATE DATABASE IF NOT EXISTS admin ON CLUSTER '{cluster}'; + +-- LOCAL TABLES +-- default database + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_beacon_blob_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon block' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The beacon block root hash' CODEC(ZSTD(1)), + `block_parent_root` FixedString(66) COMMENT 'The beacon block parent root hash' CODEC(ZSTD(1)), + `proposer_index` UInt32 COMMENT 'The validator index of the block proposer' CODEC(ZSTD(1)), + `blob_index` UInt64 COMMENT 'The index of the blob within the block' CODEC(ZSTD(1)), + `kzg_commitment` FixedString(98) COMMENT 'The KZG commitment of the blob' CODEC(ZSTD(1)), + `versioned_hash` FixedString(66) COMMENT 'The versioned hash derived from the KZG commitment (key for joining with execution_engine_get_blobs)' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block_root, blob_index) +COMMENT 'Contains beacon API blob metadata derived from block blob_kzg_commitments from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API committee payload', + `validators` Array(UInt32) COMMENT 'The validator indices in the beacon API committee payload' CODEC(ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, committee_index) +COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), + `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API event stream payload', + `attesting_validator_index` Nullable(UInt32) COMMENT 'The index of the validator attesting to the event' CODEC(ZSTD(1)), + `attesting_validator_committee_index` LowCardinality(String) COMMENT 'The committee index of the attesting validator', + `aggregation_bits` String COMMENT 'The aggregation bits of the event in the beacon API event stream payload' CODEC(ZSTD(1)), + `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `source_epoch` UInt32 COMMENT 'The source epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), + `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `target_epoch` UInt32 COMMENT 'The target epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), + `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name) +COMMENT 'Contains beacon API attestation events from each sentry client attached to a beacon node'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `blob_index` UInt64 COMMENT 'The index of blob sidecar in the beacon API event stream payload' CODEC(ZSTD(1)), + `kzg_commitment` FixedString(98) COMMENT 'The KZG commitment in the beacon API event stream payload' CODEC(ZSTD(1)), + `versioned_hash` FixedString(66) COMMENT 'The versioned hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block_root, blob_index) +COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_block_gossip_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), + `block` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block) +COMMENT 'Contains beacon API eventstream "block_gossip" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), + `block` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `execution_optimistic` Bool COMMENT 'If the attached beacon node is running in execution optimistic mode', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block) +COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number of the chain reorg event in the beacon API event stream payload', + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the reorg slot started', + `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slots between when the reorg occurred and when the sentry received the event', + `depth` UInt16 COMMENT 'The depth of the chain reorg in the beacon API event stream payload', + `old_head_block` FixedString(66) COMMENT 'The old head block root hash in the beacon API event stream payload', + `new_head_block` FixedString(66) COMMENT 'The new head block root hash in the beacon API event stream payload', + `old_head_state` FixedString(66) COMMENT 'The old head state root hash in the beacon API event stream payload', + `new_head_state` FixedString(66) COMMENT 'The new head state root hash in the beacon API event stream payload', + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', + `execution_optimistic` Bool COMMENT 'Whether the execution of the epoch was optimistic', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, old_head_block, new_head_block) +COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `aggregator_index` UInt32 COMMENT 'The validator index of the aggregator in the beacon API event stream payload', + `contribution_slot` UInt32 COMMENT 'The slot number of the contribution in the beacon API event stream payload', + `contribution_slot_start_date_time` DateTime COMMENT 'The wall clock time when the contribution slot started', + `contribution_propagation_slot_start_diff` UInt32 COMMENT 'Difference in slots between when the contribution occurred and when the sentry received the event', + `contribution_beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload', + `contribution_subcommittee_index` LowCardinality(String) COMMENT 'The subcommittee index of the contribution in the beacon API event stream payload', + `contribution_aggregation_bits` String COMMENT 'The aggregation bits of the contribution in the beacon API event stream payload', + `contribution_signature` String COMMENT 'The signature of the contribution in the beacon API event stream payload', + `contribution_epoch` UInt32 COMMENT 'The epoch number of the contribution in the beacon API event stream payload', + `contribution_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the contribution epoch started', + `selection_proof` String COMMENT 'The selection proof in the beacon API event stream payload', + `signature` String COMMENT 'The signature in the beacon API event stream payload', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(contribution_slot_start_date_time)) +ORDER BY (meta_network_name, contribution_slot_start_date_time, meta_client_name, contribution_beacon_block_root, contribution_subcommittee_index, signature) +COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_data_column_sidecar_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `column_index` UInt64 COMMENT 'The index of column in the beacon API event stream payload' CODEC(ZSTD(1)), + `kzg_commitments_count` UInt32 COMMENT 'Number of KZG commitments associated with the record' CODEC(ZSTD(1)), + `kzg_commitments` Array(FixedString(98)) COMMENT 'The KZG commitments in the beacon API event stream payload' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block_root, column_index) +COMMENT 'Contains beacon API eventstream "data_column_sidecar" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', + `block` FixedString(66) COMMENT 'The finalized block root hash in the beacon API event stream payload', + `state` FixedString(66) COMMENT 'The finalized state root hash in the beacon API event stream payload', + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `execution_optimistic` Bool COMMENT 'Whether the execution of the epoch was optimistic', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(epoch_start_date_time)) +ORDER BY (meta_network_name, epoch_start_date_time, meta_client_name, block, state) +COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload', + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', + `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time', + `block` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload', + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', + `epoch_transition` Bool COMMENT 'If the event is an epoch transition', + `execution_optimistic` Bool COMMENT 'If the attached beacon node is running in execution optimistic mode', + `previous_duty_dependent_root` FixedString(66) COMMENT 'The previous duty dependent root in the beacon API event stream payload', + `current_duty_dependent_root` FixedString(66) COMMENT 'The current duty dependent root in the beacon API event stream payload', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block, previous_duty_dependent_root, current_duty_dependent_root) +COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', + `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `validator_index` UInt32 COMMENT 'The index of the validator making the voluntary exit', + `signature` String COMMENT 'The signature of the voluntary exit in the beacon API event stream payload', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(wallclock_epoch_start_date_time)) +ORDER BY (meta_network_name, wallclock_epoch_start_date_time, meta_client_name, validator_index) +COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `proposer_validator_index` UInt32 COMMENT 'The validator index from the proposer duty payload' CODEC(ZSTD(1)), + `proposer_pubkey` String COMMENT 'The BLS public key of the validator from the proposer duty payload' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, proposer_validator_index) +COMMENT 'Contains a proposer duty from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API validator attestation data payload', + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', + `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API validator attestation data payload', + `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API validator attestation data payload', + `epoch` UInt32 COMMENT 'The epoch number in the beacon API validator attestation data payload', + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', + `source_epoch` UInt32 COMMENT 'The source epoch number in the beacon API validator attestation data payload', + `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started', + `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the beacon API validator attestation data payload', + `target_epoch` UInt32 COMMENT 'The target epoch number in the beacon API validator attestation data payload', + `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started', + `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the beacon API validator attestation data payload', + `request_date_time` DateTime COMMENT 'When the request was sent to the beacon node', + `request_duration` UInt32 COMMENT 'The request duration in milliseconds', + `request_slot_start_diff` UInt32 COMMENT 'The difference between the request_date_time and the slot_start_date_time', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, committee_index, beacon_block_root, source_root, target_root) +COMMENT 'Contains beacon API validator attestation data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload', + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the reorg slot started', + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload', + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block', + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `block_total_bytes` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload', + `block_total_bytes_compressed` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload when compressed using snappy', + `parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block', + `state_root` FixedString(66) COMMENT 'The root hash of the beacon state at this block', + `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block', + `eth1_data_block_hash` FixedString(66) COMMENT 'The block hash of the associated execution block', + `eth1_data_deposit_root` FixedString(66) COMMENT 'The root of the deposit tree in the associated execution block', + `execution_payload_block_hash` Nullable(FixedString(66)) COMMENT 'The block hash of the execution payload', + `execution_payload_block_number` Nullable(UInt32) COMMENT 'The block number of the execution payload', + `execution_payload_fee_recipient` Nullable(String) COMMENT 'The recipient of the fee for this execution payload', + `execution_payload_base_fee_per_gas` Nullable(UInt128) COMMENT 'Base fee per gas for execution payload' CODEC(ZSTD(1)), + `execution_payload_blob_gas_used` Nullable(UInt64) COMMENT 'Gas used for blobs in execution payload' CODEC(ZSTD(1)), + `execution_payload_excess_blob_gas` Nullable(UInt64) COMMENT 'Excess gas used for blobs in execution payload' CODEC(ZSTD(1)), + `execution_payload_gas_limit` Nullable(UInt64) COMMENT 'Gas limit for execution payload' CODEC(DoubleDelta, ZSTD(1)), + `execution_payload_gas_used` Nullable(UInt64) COMMENT 'Gas used for execution payload' CODEC(ZSTD(1)), + `execution_payload_state_root` FixedString(66) COMMENT 'The state root of the execution payload', + `execution_payload_parent_hash` FixedString(66) COMMENT 'The parent hash of the execution payload', + `execution_payload_transactions_count` Nullable(UInt32) COMMENT 'The transaction count of the execution payload', + `execution_payload_transactions_total_bytes` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload', + `execution_payload_transactions_total_bytes_compressed` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload when compressed using snappy', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block_root, parent_root, state_root) +COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v3_validator_block_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number within the payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `block_total_bytes` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload' CODEC(ZSTD(1)), + `block_total_bytes_compressed` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload when compressed using snappy' CODEC(ZSTD(1)), + `consensus_payload_value` Nullable(UInt64) COMMENT 'Consensus rewards paid to the proposer for this block, in Wei. Use to determine relative value of consensus blocks.' CODEC(ZSTD(1)), + `execution_payload_value` Nullable(UInt64) COMMENT 'Execution payload value in Wei. Use to determine relative value of execution payload.' CODEC(ZSTD(1)), + `execution_payload_block_number` UInt32 COMMENT 'The block number of the execution payload', + `execution_payload_base_fee_per_gas` Nullable(UInt128) COMMENT 'Base fee per gas for execution payload' CODEC(ZSTD(1)), + `execution_payload_blob_gas_used` Nullable(UInt64) COMMENT 'Gas used for blobs in execution payload' CODEC(ZSTD(1)), + `execution_payload_excess_blob_gas` Nullable(UInt64) COMMENT 'Excess gas used for blobs in execution payload' CODEC(ZSTD(1)), + `execution_payload_gas_limit` Nullable(UInt64) COMMENT 'Gas limit for execution payload' CODEC(DoubleDelta, ZSTD(1)), + `execution_payload_gas_used` Nullable(UInt64) COMMENT 'Gas used for execution payload' CODEC(ZSTD(1)), + `execution_payload_transactions_count` Nullable(UInt32) COMMENT 'The transaction count of the execution payload' CODEC(ZSTD(1)), + `execution_payload_transactions_total_bytes` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload' CODEC(ZSTD(1)), + `execution_payload_transactions_total_bytes_compressed` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload when compressed using snappy' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, event_date_time) +COMMENT 'Contains beacon API /eth/v3/validator/blocks/{slot} data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_slot_local ON CLUSTER '{cluster}' +( + `slot` UInt32 COMMENT 'Slot number' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `blocks` AggregateFunction(sum, UInt16) COMMENT 'The number of beacon blocks seen in the slot' CODEC(ZSTD(1)), + `attestations` AggregateFunction(sum, UInt32) COMMENT 'The number of attestations seen in the slot' CODEC(ZSTD(1)) +) +ENGINE = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, slot) +COMMENT 'Contains beacon API slot data from each sentry client attached to a beacon node'; + +CREATE TABLE IF NOT EXISTS default.beacon_block_classification_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block classification', + `slot` UInt32 COMMENT 'The slot number from beacon block classification', + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block classification' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `best_guess_single` LowCardinality(String) COMMENT 'The best guess of the client that generated the beacon block', + `best_guess_multi` LowCardinality(String) COMMENT 'The best guess of the clients that generated the beacon block. This value will typically equal the best_guess_single value, but when multiple clients have high probabilities, this value will have multiple eg. "prysm or lighthouse"', + `client_probability_uncertain` Float32 COMMENT 'The probability that the client that generated the beacon block is uncertain' CODEC(ZSTD(1)), + `client_probability_prysm` Float32 COMMENT 'The probability that the client that generated the beacon block is Prysm' CODEC(ZSTD(1)), + `client_probability_teku` Float32 COMMENT 'The probability that the client that generated the beacon block is Teku' CODEC(ZSTD(1)), + `client_probability_nimbus` Float32 COMMENT 'The probability that the client that generated the beacon block is Nimbus' CODEC(ZSTD(1)), + `client_probability_lodestar` Float32 COMMENT 'The probability that the client that generated the beacon block is Lodestar' CODEC(ZSTD(1)), + `client_probability_grandine` Float32 COMMENT 'The probability that the client that generated the beacon block is Grandine' CODEC(ZSTD(1)), + `client_probability_lighthouse` Float32 COMMENT 'The probability that the client that generated the beacon block is Lighthouse' CODEC(ZSTD(1)), + `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', + `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', + `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', + `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', + `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, proposer_index) +COMMENT 'Contains beacon block classification for a given slot. This is a best guess based on the client probabilities of the proposer. This is not guaranteed to be correct.'; + +CREATE TABLE IF NOT EXISTS default.blob_submitter_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `address` FixedString(66) COMMENT 'Ethereum address of the blob submitter' CODEC(ZSTD(1)), + `name` String COMMENT 'Name of the blob submitter' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY meta_network_name +ORDER BY (meta_network_name, address) +COMMENT 'Contains blob submitter address to name mappings.'; + +CREATE TABLE IF NOT EXISTS default.block_native_mempool_transaction_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `detecttime` DateTime64(3) COMMENT 'Timestamp that the transaction was detected in mempool' CODEC(DoubleDelta, ZSTD(1)), + `hash` FixedString(66) COMMENT 'Unique identifier hash for a given transaction' CODEC(ZSTD(1)), + `status` LowCardinality(String) COMMENT 'Status of the transaction', + `region` LowCardinality(String) COMMENT 'The geographic region for the node that detected the transaction', + `reorg` Nullable(FixedString(66)) COMMENT 'If there was a reorg, refers to the blockhash of the reorg' CODEC(ZSTD(1)), + `replace` Nullable(FixedString(66)) COMMENT 'If the transaction was replaced (speedup/cancel), the transaction hash of the replacement' CODEC(ZSTD(1)), + `curblocknumber` Nullable(UInt64) COMMENT 'The block number the event was detected in' CODEC(ZSTD(1)), + `failurereason` Nullable(String) COMMENT 'If a transaction failed, this field provides contextual information' CODEC(ZSTD(1)), + `blockspending` Nullable(UInt64) COMMENT 'If a transaction was finalized (confirmed, failed), this refers to the number of blocks that the transaction was waiting to get on-chain' CODEC(ZSTD(1)), + `timepending` Nullable(UInt64) COMMENT 'If a transaction was finalized (confirmed, failed), this refers to the time in milliseconds that the transaction was waiting to get on-chain' CODEC(ZSTD(1)), + `nonce` UInt64 COMMENT 'A unique number which counts the number of transactions sent from a given address' CODEC(ZSTD(1)), + `gas` UInt64 COMMENT 'The maximum number of gas units allowed for the transaction' CODEC(ZSTD(1)), + `gasprice` UInt128 COMMENT 'The price offered to the miner/validator per unit of gas. Denominated in wei' CODEC(ZSTD(1)), + `value` UInt128 COMMENT 'The amount of ETH transferred or sent to contract. Denominated in wei' CODEC(ZSTD(1)), + `toaddress` Nullable(FixedString(42)) COMMENT 'The destination of a given transaction' CODEC(ZSTD(1)), + `fromaddress` FixedString(42) COMMENT 'The source/initiator of a given transaction' CODEC(ZSTD(1)), + `datasize` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), + `data4bytes` Nullable(FixedString(10)) COMMENT 'The first 4 bytes of the call data of the transaction' CODEC(ZSTD(1)), + `network` LowCardinality(String) COMMENT 'The specific Ethereum network used', + `type` UInt8 COMMENT '"Post EIP-1559, this indicates how the gas parameters are submitted to the network: - type 0 - legacy - type 1 - usage of access lists according to EIP-2930 - type 2 - using maxpriorityfeepergas and maxfeepergas"' CODEC(ZSTD(1)), + `maxpriorityfeepergas` Nullable(UInt128) COMMENT 'The maximum value for a tip offered to the miner/validator per unit of gas. The actual tip paid can be lower if (maxfee - basefee) < maxpriorityfee. Denominated in wei' CODEC(ZSTD(1)), + `maxfeepergas` Nullable(UInt128) COMMENT 'The maximum value for the transaction fee (including basefee and tip) offered to the miner/validator per unit of gas. Denominated in wei' CODEC(ZSTD(1)), + `basefeepergas` Nullable(UInt128) COMMENT 'The fee per unit of gas paid and burned for the curblocknumber. This fee is algorithmically determined. Denominated in wei' CODEC(ZSTD(1)), + `dropreason` Nullable(String) COMMENT 'If the transaction was dropped from the mempool, this describes the contextual reason for the drop' CODEC(ZSTD(1)), + `rejectionreason` Nullable(String) COMMENT 'If the transaction was rejected from the mempool, this describes the contextual reason for the rejection' CODEC(ZSTD(1)), + `stuck` Bool COMMENT 'A transaction was detected in the queued area of the mempool and is not eligible for inclusion in a block' CODEC(ZSTD(1)), + `gasused` Nullable(UInt64) COMMENT 'If the transaction was published on-chain, this value indicates the amount of gas that was actually consumed. Denominated in wei' CODEC(ZSTD(1)) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (network, toYYYYMM(detecttime)) +ORDER BY (network, detecttime, hash, fromaddress, nonce, gas) +COMMENT 'Contains transactions from block native mempool dataset'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block' CODEC(ZSTD(1)), + `versioned_hash` FixedString(66) COMMENT 'The versioned hash in the beacon API event stream payload' CODEC(ZSTD(1)), + `kzg_commitment` FixedString(98) COMMENT 'The KZG commitment in the blob sidecar payload' CODEC(ZSTD(1)), + `kzg_proof` FixedString(98) COMMENT 'The KZG proof in the blob sidecar payload' CODEC(ZSTD(1)), + `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), + `blob_index` UInt64 COMMENT 'The index of blob sidecar in the blob sidecar payload' CODEC(ZSTD(1)), + `blob_size` UInt32 COMMENT 'The total bytes of the blob' CODEC(ZSTD(1)), + `blob_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the blob in bytes' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, blob_index) +COMMENT 'Contains a blob sidecar from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `attestation_1_attesting_indices` Array(UInt32) COMMENT 'The attesting indices from the first attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_1_signature` String COMMENT 'The signature from the first attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_1_data_beacon_block_root` FixedString(66) COMMENT 'The beacon block root from the first attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_1_data_slot` UInt32 COMMENT 'The slot number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `attestation_1_data_index` UInt32 COMMENT 'The attestor index from the first attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_1_data_source_epoch` UInt32 COMMENT 'The source epoch number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `attestation_1_data_source_root` FixedString(66) COMMENT 'The source root from the first attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_1_data_target_epoch` UInt32 COMMENT 'The target epoch number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `attestation_1_data_target_root` FixedString(66) COMMENT 'The target root from the first attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_2_attesting_indices` Array(UInt32) COMMENT 'The attesting indices from the second attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_2_signature` String COMMENT 'The signature from the second attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_2_data_beacon_block_root` FixedString(66) COMMENT 'The beacon block root from the second attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_2_data_slot` UInt32 COMMENT 'The slot number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `attestation_2_data_index` UInt32 COMMENT 'The attestor index from the second attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_2_data_source_epoch` UInt32 COMMENT 'The source epoch number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `attestation_2_data_source_root` FixedString(66) COMMENT 'The source root from the second attestation in the slashing payload' CODEC(ZSTD(1)), + `attestation_2_data_target_epoch` UInt32 COMMENT 'The target epoch number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `attestation_2_data_target_root` FixedString(66) COMMENT 'The target root from the second attestation in the slashing payload' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, attestation_1_attesting_indices, attestation_2_attesting_indices, attestation_1_data_slot, attestation_2_data_slot, attestation_1_data_beacon_block_root, attestation_2_data_beacon_block_root) +COMMENT 'Contains attester slashing from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `exchanging_message_validator_index` UInt32 COMMENT 'The validator index from the exchanging message' CODEC(ZSTD(1)), + `exchanging_message_from_bls_pubkey` String COMMENT 'The BLS public key from the exchanging message' CODEC(ZSTD(1)), + `exchanging_message_to_execution_address` FixedString(42) COMMENT 'The execution address from the exchanging message' CODEC(ZSTD(1)), + `exchanging_signature` String COMMENT 'The signature for the exchanging message' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, exchanging_message_validator_index, exchanging_message_from_bls_pubkey, exchanging_message_to_execution_address) +COMMENT 'Contains bls to execution change from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `deposit_proof` Array(String) COMMENT 'The proof of the deposit data' CODEC(ZSTD(1)), + `deposit_data_pubkey` String COMMENT 'The BLS public key of the validator from the deposit data' CODEC(ZSTD(1)), + `deposit_data_withdrawal_credentials` FixedString(66) COMMENT 'The withdrawal credentials of the validator from the deposit data' CODEC(ZSTD(1)), + `deposit_data_amount` UInt128 COMMENT 'The amount of the deposit from the deposit data' CODEC(ZSTD(1)), + `deposit_data_signature` String COMMENT 'The signature of the deposit data' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, deposit_data_pubkey, deposit_proof) +COMMENT 'Contains a deposit from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `position` UInt32 COMMENT 'The position of the transaction in the beacon block' CODEC(DoubleDelta, ZSTD(1)), + `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), + `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), + `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), + `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), + `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), + `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), + `gas_tip_cap` Nullable(UInt128) COMMENT 'The priority fee (tip) the user has set for the transaction' CODEC(ZSTD(1)), + `gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), + `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), + `type` UInt8 COMMENT 'The type of the transaction' CODEC(ZSTD(1)), + `size` UInt32 COMMENT 'The size of the transaction data in bytes' CODEC(ZSTD(1)), + `call_data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), + `blob_gas` Nullable(UInt64) COMMENT 'The maximum gas provided for the blob transaction execution' CODEC(ZSTD(1)), + `blob_gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), + `blob_hashes` Array(String) COMMENT 'The hashes of the blob commitments for blob transactions' CODEC(ZSTD(1)), + `blob_sidecars_size` Nullable(UInt32) COMMENT 'The total size of the sidecars for blob transactions in bytes' CODEC(ZSTD(1)), + `blob_sidecars_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the sidecars for blob transactions in bytes' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, position, hash, nonce) +COMMENT 'Contains execution transaction from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `block_total_bytes` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload' CODEC(ZSTD(1)), + `block_total_bytes_compressed` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload when compressed using snappy' CODEC(ZSTD(1)), + `parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block' CODEC(ZSTD(1)), + `state_root` FixedString(66) COMMENT 'The root hash of the beacon state at this block' CODEC(ZSTD(1)), + `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), + `eth1_data_block_hash` FixedString(66) COMMENT 'The block hash of the associated execution block' CODEC(ZSTD(1)), + `eth1_data_deposit_root` FixedString(66) COMMENT 'The root of the deposit tree in the associated execution block' CODEC(ZSTD(1)), + `execution_payload_block_hash` Nullable(FixedString(66)) COMMENT 'The block hash of the execution payload' CODEC(ZSTD(1)), + `execution_payload_block_number` Nullable(UInt32) COMMENT 'The block number of the execution payload' CODEC(DoubleDelta, ZSTD(1)), + `execution_payload_fee_recipient` Nullable(String) COMMENT 'The recipient of the fee for this execution payload' CODEC(ZSTD(1)), + `execution_payload_base_fee_per_gas` Nullable(UInt128) COMMENT 'Base fee per gas for execution payload' CODEC(ZSTD(1)), + `execution_payload_blob_gas_used` Nullable(UInt64) COMMENT 'Gas used for blobs in execution payload' CODEC(ZSTD(1)), + `execution_payload_excess_blob_gas` Nullable(UInt64) COMMENT 'Excess gas used for blobs in execution payload' CODEC(ZSTD(1)), + `execution_payload_gas_limit` Nullable(UInt64) COMMENT 'Gas limit for execution payload' CODEC(DoubleDelta, ZSTD(1)), + `execution_payload_gas_used` Nullable(UInt64) COMMENT 'Gas used for execution payload' CODEC(ZSTD(1)), + `execution_payload_state_root` Nullable(FixedString(66)) COMMENT 'The state root of the execution payload' CODEC(ZSTD(1)), + `execution_payload_parent_hash` Nullable(FixedString(66)) COMMENT 'The parent hash of the execution payload' CODEC(ZSTD(1)), + `execution_payload_transactions_count` Nullable(UInt32) COMMENT 'The transaction count of the execution payload' CODEC(ZSTD(1)), + `execution_payload_transactions_total_bytes` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload' CODEC(ZSTD(1)), + `execution_payload_transactions_total_bytes_compressed` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload when compressed using snappy' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time) +COMMENT 'Contains beacon block from a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `signed_header_1_message_slot` UInt32 COMMENT 'The slot number from the first signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `signed_header_1_message_proposer_index` UInt32 COMMENT 'The proposer index from the first signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `signed_header_1_message_body_root` FixedString(66) COMMENT 'The body root from the first signed header in the slashing payload' CODEC(ZSTD(1)), + `signed_header_1_message_parent_root` FixedString(66) COMMENT 'The parent root from the first signed header in the slashing payload' CODEC(ZSTD(1)), + `signed_header_1_message_state_root` FixedString(66) COMMENT 'The state root from the first signed header in the slashing payload' CODEC(ZSTD(1)), + `signed_header_1_signature` String COMMENT 'The signature for the first signed header in the slashing payload' CODEC(ZSTD(1)), + `signed_header_2_message_slot` UInt32 COMMENT 'The slot number from the second signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `signed_header_2_message_proposer_index` UInt32 COMMENT 'The proposer index from the second signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), + `signed_header_2_message_body_root` FixedString(66) COMMENT 'The body root from the second signed header in the slashing payload' CODEC(ZSTD(1)), + `signed_header_2_message_parent_root` FixedString(66) COMMENT 'The parent root from the second signed header in the slashing payload' CODEC(ZSTD(1)), + `signed_header_2_message_state_root` FixedString(66) COMMENT 'The state root from the second signed header in the slashing payload' CODEC(ZSTD(1)), + `signed_header_2_signature` String COMMENT 'The signature for the second signed header in the slashing payload' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, signed_header_1_message_slot, signed_header_2_message_slot, signed_header_1_message_proposer_index, signed_header_2_message_proposer_index, signed_header_1_message_body_root, signed_header_2_message_body_root) +COMMENT 'Contains proposer slashing from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_sync_aggregate_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon chain' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon chain' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `sync_committee_period` UInt64 COMMENT 'The sync committee period number (epoch / 256)' CODEC(DoubleDelta, ZSTD(1)), + `sync_committee_bits` String COMMENT 'Raw 512-bit bitvector as hex string' CODEC(ZSTD(1)), + `sync_committee_signature` String COMMENT 'Aggregated signature from participating validators' CODEC(ZSTD(1)), + `validators_participated` Array(UInt32) COMMENT 'Validator indices that participated (voted)' CODEC(ZSTD(1)), + `validators_missed` Array(UInt32) COMMENT 'Validator indices that missed (did not vote)' CODEC(ZSTD(1)), + `participation_count` UInt16 COMMENT 'Number of validators that participated (0-512)' CODEC(DoubleDelta, ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, slot) +COMMENT 'Contains canonical beacon block sync aggregate data with expanded validator participation.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `voluntary_exit_message_epoch` UInt32 COMMENT 'The epoch number from the exit message' CODEC(DoubleDelta, ZSTD(1)), + `voluntary_exit_message_validator_index` UInt32 COMMENT 'The validator index from the exit message' CODEC(ZSTD(1)), + `voluntary_exit_signature` String COMMENT 'The signature of the exit message' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, voluntary_exit_message_epoch, voluntary_exit_message_validator_index) +COMMENT 'Contains a voluntary exit from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), + `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', + `withdrawal_index` UInt32 COMMENT 'The index of the withdrawal' CODEC(ZSTD(1)), + `withdrawal_validator_index` UInt32 COMMENT 'The validator index from the withdrawal data' CODEC(ZSTD(1)), + `withdrawal_address` FixedString(42) COMMENT 'The address of the account that is the withdrawal recipient' CODEC(ZSTD(1)), + `withdrawal_amount` UInt128 COMMENT 'The amount of the withdrawal from the withdrawal data' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, withdrawal_index, withdrawal_validator_index) +COMMENT 'Contains a withdrawal from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_committee_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API committee payload', + `validators` Array(UInt32) COMMENT 'The validator indices in the beacon API committee payload' CODEC(ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, committee_index) +COMMENT 'Contains canonical beacon API /eth/v1/beacon/committees data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_slot` UInt32 COMMENT 'The slot number of the block containing the attestation' CODEC(DoubleDelta, ZSTD(1)), + `block_slot_start_date_time` DateTime COMMENT 'The wall clock time when the block slot started' CODEC(DoubleDelta, ZSTD(1)), + `block_epoch` UInt32 COMMENT 'The epoch number of the block containing the attestation' CODEC(DoubleDelta, ZSTD(1)), + `block_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the block epoch started' CODEC(DoubleDelta, ZSTD(1)), + `position_in_block` UInt32 COMMENT 'The position of the attestation in the block' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'The root of the block containing the attestation' CODEC(ZSTD(1)), + `validators` Array(UInt32) COMMENT 'Array of validator indices participating in the attestation' CODEC(ZSTD(1)), + `committee_index` LowCardinality(String) COMMENT 'The index of the committee making the attestation', + `beacon_block_root` FixedString(66) COMMENT 'The root of the beacon block being attested to' CODEC(ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number being attested to' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), + `source_epoch` UInt32 COMMENT 'The source epoch referenced in the attestation' CODEC(DoubleDelta, ZSTD(1)), + `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), + `source_root` FixedString(66) COMMENT 'The root of the source checkpoint in the attestation' CODEC(ZSTD(1)), + `target_epoch` UInt32 COMMENT 'The target epoch referenced in the attestation' CODEC(DoubleDelta, ZSTD(1)), + `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), + `target_root` FixedString(66) COMMENT 'The root of the target checkpoint in the attestation' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, block_root, block_slot, position_in_block, beacon_block_root, slot, committee_index, source_root, target_root) +COMMENT 'Contains elaborated attestations from beacon blocks.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'The slot number for which the proposer duty is assigned' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number containing the slot' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `proposer_validator_index` UInt32 COMMENT 'The validator index of the proposer for the slot' CODEC(ZSTD(1)), + `proposer_pubkey` String COMMENT 'The public key of the validator proposer' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, proposer_validator_index, proposer_pubkey) +COMMENT 'Contains a proposer duty from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_sync_committee_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number for when the sync committee is active' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `sync_committee_period` UInt64 COMMENT 'The sync committee period number' CODEC(DoubleDelta, ZSTD(1)), + `validator_aggregates` Array(Array(UInt32)) COMMENT 'The validator indices grouped by subcommittee (64 groups of 8)' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(epoch_start_date_time)) +ORDER BY (meta_network_name, epoch_start_date_time, sync_committee_period) +COMMENT 'Contains canonical beacon API /eth/v1/beacon/states/{state_id}/sync_committees data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_validators_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `index` UInt32 COMMENT 'The index of the validator' CODEC(DoubleDelta, ZSTD(1)), + `balance` Nullable(UInt64) COMMENT 'The balance of the validator' CODEC(T64, ZSTD(1)), + `status` LowCardinality(String) COMMENT 'The status of the validator', + `effective_balance` Nullable(UInt64) COMMENT 'The effective balance of the validator' CODEC(ZSTD(1)), + `slashed` Bool COMMENT 'Whether the validator is slashed', + `activation_epoch` Nullable(UInt64) COMMENT 'The epoch when the validator was activated' CODEC(ZSTD(1)), + `activation_eligibility_epoch` Nullable(UInt64) COMMENT 'The epoch when the validator was activated' CODEC(ZSTD(1)), + `exit_epoch` Nullable(UInt64) COMMENT 'The epoch when the validator exited' CODEC(ZSTD(1)), + `withdrawable_epoch` Nullable(UInt64) COMMENT 'The epoch when the validator can withdraw' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(epoch_start_date_time)) +ORDER BY (meta_network_name, epoch_start_date_time, index, status) +COMMENT 'Contains a validator state for an epoch.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_validators_pubkeys_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `index` UInt32 COMMENT 'The index of the validator' CODEC(ZSTD(1)), + `pubkey` String COMMENT 'The public key of the validator' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY meta_network_name +ORDER BY (meta_network_name, index, pubkey) +COMMENT 'Contains a validator state for an epoch.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_validators_withdrawal_credentials_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `index` UInt32 COMMENT 'The index of the validator' CODEC(ZSTD(1)), + `withdrawal_credentials` String COMMENT 'The withdrawal credentials of the validator' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY meta_network_name +ORDER BY (meta_network_name, index, withdrawal_credentials); + +CREATE TABLE IF NOT EXISTS default.canonical_execution_address_appearances_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the address appearance' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the address appearance within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `address` String COMMENT 'The address of the address appearance' CODEC(ZSTD(1)), + `relationship` LowCardinality(String) COMMENT 'The relationship of the address to the transaction', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution address appearance data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_balance_diffs_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the balance diff' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the balance diff within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `address` String COMMENT 'The address of the balance diff' CODEC(ZSTD(1)), + `from_value` UInt256 COMMENT 'The from value of the balance diff' CODEC(ZSTD(1)), + `to_value` UInt256 COMMENT 'The to value of the balance diff' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution balance diff data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_balance_reads_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the balance read' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the balance read within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `address` String COMMENT 'The address of the balance read' CODEC(ZSTD(1)), + `balance` UInt256 COMMENT 'The balance that was read' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution balance read data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_block_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_date_time` DateTime64(3) COMMENT 'The block timestamp' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `block_hash` FixedString(66) COMMENT 'The block hash' CODEC(ZSTD(1)), + `author` Nullable(String) COMMENT 'The block author' CODEC(ZSTD(1)), + `gas_used` Nullable(UInt64) COMMENT 'The block gas used' CODEC(DoubleDelta, ZSTD(1)), + `gas_limit` UInt64 COMMENT 'The block gas limit' CODEC(DoubleDelta, ZSTD(1)), + `extra_data` Nullable(String) COMMENT 'The block extra data in hex' CODEC(ZSTD(1)), + `extra_data_string` Nullable(String) COMMENT 'The block extra data in UTF-8 string' CODEC(ZSTD(1)), + `base_fee_per_gas` Nullable(UInt64) COMMENT 'The block base fee per gas' CODEC(DoubleDelta, ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number) +COMMENT 'Contains canonical execution block data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_contracts_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash that created the contract' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the contract creation within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `create_index` UInt32 COMMENT 'The create index' CODEC(DoubleDelta, ZSTD(1)), + `contract_address` String COMMENT 'The contract address' CODEC(ZSTD(1)), + `deployer` String COMMENT 'The address of the contract deployer' CODEC(ZSTD(1)), + `factory` String COMMENT 'The address of the factory that deployed the contract' CODEC(ZSTD(1)), + `init_code` String COMMENT 'The initialization code of the contract' CODEC(ZSTD(1)), + `code` Nullable(String) COMMENT 'The code of the contract' CODEC(ZSTD(1)), + `init_code_hash` String COMMENT 'The hash of the initialization code' CODEC(ZSTD(1)), + `n_init_code_bytes` UInt32 COMMENT 'Number of bytes in the initialization code' CODEC(DoubleDelta, ZSTD(1)), + `n_code_bytes` UInt32 COMMENT 'Number of bytes in the contract code' CODEC(DoubleDelta, ZSTD(1)), + `code_hash` String COMMENT 'The hash of the contract code' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution contract data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_erc20_transfers_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the transfer within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `log_index` UInt64 COMMENT 'The log index in the block' CODEC(DoubleDelta, ZSTD(1)), + `erc20` String COMMENT 'The erc20 address' CODEC(ZSTD(1)), + `from_address` String COMMENT 'The from address' CODEC(ZSTD(1)), + `to_address` String COMMENT 'The to address' CODEC(ZSTD(1)), + `value` UInt256 COMMENT 'The value of the transfer' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution erc20 transfer data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_erc721_transfers_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the transfer within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `log_index` UInt64 COMMENT 'The log index in the block' CODEC(DoubleDelta, ZSTD(1)), + `erc721` String COMMENT 'The erc20 address' CODEC(ZSTD(1)), + `from_address` String COMMENT 'The from address' CODEC(ZSTD(1)), + `to_address` String COMMENT 'The to address' CODEC(ZSTD(1)), + `token` UInt256 COMMENT 'The token id' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution erc721 transfer data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_four_byte_counts_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), + `signature` String COMMENT 'The signature of the four byte count' CODEC(ZSTD(1)), + `size` UInt64 COMMENT 'The size of the four byte count' CODEC(ZSTD(1)), + `count` UInt64 COMMENT 'The count of the four byte count' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash) +COMMENT 'Contains canonical execution four byte count data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_logs_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash associated with the log' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the log within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `log_index` UInt32 COMMENT 'The log index within the block' CODEC(DoubleDelta, ZSTD(1)), + `address` String COMMENT 'The address associated with the log' CODEC(ZSTD(1)), + `topic0` String COMMENT 'The first topic of the log' CODEC(ZSTD(1)), + `topic1` Nullable(String) COMMENT 'The second topic of the log' CODEC(ZSTD(1)), + `topic2` Nullable(String) COMMENT 'The third topic of the log' CODEC(ZSTD(1)), + `topic3` Nullable(String) COMMENT 'The fourth topic of the log' CODEC(ZSTD(1)), + `data` Nullable(String) COMMENT 'The data associated with the log' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution logs data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_native_transfers_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the transfer within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `transfer_index` UInt64 COMMENT 'The transfer index' CODEC(DoubleDelta, ZSTD(1)), + `from_address` String COMMENT 'The from address' CODEC(ZSTD(1)), + `to_address` String COMMENT 'The to address' CODEC(ZSTD(1)), + `value` UInt256 COMMENT 'The value of the approval' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution native transfer data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_nonce_diffs_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the nonce diff' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the nonce diff within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `address` String COMMENT 'The address of the nonce diff' CODEC(ZSTD(1)), + `from_value` UInt64 COMMENT 'The from value of the nonce diff' CODEC(ZSTD(1)), + `to_value` UInt64 COMMENT 'The to value of the nonce diff' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution nonce diff data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_nonce_reads_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the nonce read' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the nonce read within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `address` String COMMENT 'The address of the nonce read' CODEC(ZSTD(1)), + `nonce` UInt64 COMMENT 'The nonce that was read' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution nonce read data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_storage_diffs_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash associated with the storage diff' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the storage diff within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `address` String COMMENT 'The address associated with the storage diff' CODEC(ZSTD(1)), + `slot` String COMMENT 'The storage slot key' CODEC(ZSTD(1)), + `from_value` String COMMENT 'The original value before the storage diff' CODEC(ZSTD(1)), + `to_value` String COMMENT 'The new value after the storage diff' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution storage diffs data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_storage_reads_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash associated with the storage read' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the storage read within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `contract_address` String COMMENT 'The contract address associated with the storage read' CODEC(ZSTD(1)), + `slot` String COMMENT 'The storage slot key' CODEC(ZSTD(1)), + `value` String COMMENT 'The value read from the storage slot' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution storage reads data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_traces_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), + `internal_index` UInt32 COMMENT 'The internal index of the trace within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `action_from` String COMMENT 'The from address of the action' CODEC(ZSTD(1)), + `action_to` Nullable(String) COMMENT 'The to address of the action' CODEC(ZSTD(1)), + `action_value` UInt256 COMMENT 'The value of the action' CODEC(ZSTD(1)), + `action_gas` UInt64 COMMENT 'The gas provided for the action' CODEC(DoubleDelta, ZSTD(1)), + `action_input` Nullable(String) COMMENT 'The input data for the action' CODEC(ZSTD(1)), + `action_call_type` LowCardinality(String) COMMENT 'The call type of the action' CODEC(ZSTD(1)), + `action_init` Nullable(String) COMMENT 'The initialization code for the action' CODEC(ZSTD(1)), + `action_reward_type` String COMMENT 'The reward type for the action' CODEC(ZSTD(1)), + `action_type` LowCardinality(String) COMMENT 'The type of the action' CODEC(ZSTD(1)), + `result_gas_used` UInt64 COMMENT 'The gas used in the result' CODEC(DoubleDelta, ZSTD(1)), + `result_output` Nullable(String) COMMENT 'The output of the result' CODEC(ZSTD(1)), + `result_code` Nullable(String) COMMENT 'The code returned in the result' CODEC(ZSTD(1)), + `result_address` Nullable(String) COMMENT 'The address returned in the result' CODEC(ZSTD(1)), + `trace_address` Nullable(String) COMMENT 'The trace address' CODEC(ZSTD(1)), + `subtraces` UInt32 COMMENT 'The number of subtraces' CODEC(DoubleDelta, ZSTD(1)), + `error` Nullable(String) COMMENT 'The error, if any, in the trace' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash, internal_index) +COMMENT 'Contains canonical execution traces data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_transaction_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), + `nonce` UInt64 COMMENT 'The transaction nonce' CODEC(ZSTD(1)), + `from_address` String COMMENT 'The transaction from address' CODEC(ZSTD(1)), + `to_address` Nullable(String) COMMENT 'The transaction to address' CODEC(ZSTD(1)), + `value` UInt256 COMMENT 'The transaction value in float64' CODEC(ZSTD(1)), + `input` Nullable(String) COMMENT 'The transaction input in hex' CODEC(ZSTD(1)), + `gas_limit` UInt64 COMMENT 'The transaction gas limit' CODEC(ZSTD(1)), + `gas_used` UInt64 COMMENT 'The transaction gas used' CODEC(ZSTD(1)), + `gas_price` UInt128 COMMENT 'The transaction gas price' CODEC(ZSTD(1)), + `transaction_type` UInt8 COMMENT 'The transaction type' CODEC(ZSTD(1)), + `max_priority_fee_per_gas` UInt64 COMMENT 'The transaction max priority fee per gas' CODEC(ZSTD(1)), + `max_fee_per_gas` UInt64 COMMENT 'The transaction max fee per gas' CODEC(ZSTD(1)), + `success` Bool COMMENT 'The transaction success' CODEC(ZSTD(1)), + `n_input_bytes` UInt32 COMMENT 'The transaction input bytes' CODEC(ZSTD(1)), + `n_input_zero_bytes` UInt32 COMMENT 'The transaction input zero bytes' CODEC(ZSTD(1)), + `n_input_nonzero_bytes` UInt32 COMMENT 'The transaction input nonzero bytes' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, transaction_hash) +COMMENT 'Contains canonical execution transaction data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_transaction_structlog_agg_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction position in the block' CODEC(DoubleDelta, ZSTD(1)), + `call_frame_id` UInt32 COMMENT 'Sequential frame ID within the transaction (0=root)' CODEC(DoubleDelta, ZSTD(1)), + `parent_call_frame_id` Nullable(UInt32) COMMENT 'Parent frame ID (NULL for root frame)' CODEC(ZSTD(1)), + `call_frame_path` Array(UInt32) COMMENT 'Path of frame IDs from root to current frame' CODEC(ZSTD(1)), + `depth` UInt32 COMMENT 'Call nesting depth (0=root)' CODEC(DoubleDelta, ZSTD(1)), + `target_address` Nullable(String) COMMENT 'Contract address being called' CODEC(ZSTD(1)), + `call_type` LowCardinality(String) COMMENT 'Call type: CALL/DELEGATECALL/STATICCALL/CALLCODE/CREATE/CREATE2 (empty for root)', + `operation` LowCardinality(String) COMMENT 'Opcode name for per-opcode rows, empty string for frame summary rows', + `opcode_count` UInt64 COMMENT 'Number of opcodes (total for summary row, count for per-opcode row)' CODEC(ZSTD(1)), + `error_count` UInt64 COMMENT 'Number of errors' CODEC(ZSTD(1)), + `gas` UInt64 COMMENT 'Gas consumed: SUM(gas_self) for per-opcode, frame self gas for summary' CODEC(ZSTD(1)), + `gas_cumulative` UInt64 COMMENT 'Cumulative gas: SUM(gas_used) for per-opcode, frame total for summary' CODEC(ZSTD(1)), + `min_depth` UInt32 COMMENT 'Minimum depth where opcode appeared (per-opcode rows)' CODEC(DoubleDelta, ZSTD(1)), + `max_depth` UInt32 COMMENT 'Maximum depth where opcode appeared (per-opcode rows)' CODEC(DoubleDelta, ZSTD(1)), + `memory_words_sum_before` UInt64 DEFAULT 0 COMMENT 'SUM(ceil(memory_bytes/32)) before each opcode executes. Used with sq_sum to compute memory expansion gas.' CODEC(ZSTD(1)), + `memory_words_sum_after` UInt64 DEFAULT 0 COMMENT 'SUM(ceil(memory_bytes/32)) after each opcode executes.' CODEC(ZSTD(1)), + `memory_words_sq_sum_before` UInt64 DEFAULT 0 COMMENT 'SUM(words_before²). With sum_before, enables exact memory gas via E[cost(after)] - E[cost(before)].' CODEC(ZSTD(1)), + `memory_words_sq_sum_after` UInt64 DEFAULT 0 COMMENT 'SUM(words_after²). With sum_after, enables exact memory gas via E[cost(after)] - E[cost(before)].' CODEC(ZSTD(1)), + `memory_expansion_gas` UInt64 DEFAULT 0 COMMENT 'SUM(memory_expansion_gas). Exact per-opcode memory expansion cost, pre-computed to avoid intDiv rounding in SQL reconstruction.' CODEC(ZSTD(1)), + `cold_access_count` UInt64 DEFAULT 0 COMMENT 'Number of cold storage/account accesses (EIP-2929). cold_gas = cold_count * (cold_cost - warm_cost).' CODEC(ZSTD(1)), + `gas_refund` Nullable(UInt64) COMMENT 'Gas refund (root summary row only)' CODEC(ZSTD(1)), + `intrinsic_gas` Nullable(UInt64) COMMENT 'Intrinsic gas (root summary row only, computed)' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 201600)) +ORDER BY (meta_network_name, block_number, transaction_hash, call_frame_id, operation) +COMMENT 'Aggregated EVM execution data. Summary rows (operation="") contain frame metadata. Per-opcode rows contain aggregated gas/count per (frame, opcode).'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), + `transaction_index` UInt64 COMMENT 'The transaction position in the block' CODEC(DoubleDelta, ZSTD(1)), + `transaction_gas` UInt64 COMMENT 'The transaction gas' CODEC(DoubleDelta, ZSTD(1)), + `transaction_failed` Bool COMMENT 'The transaction failed' CODEC(ZSTD(1)), + `transaction_return_value` Nullable(String) COMMENT 'The transaction return value' CODEC(ZSTD(1)), + `index` UInt32 COMMENT 'The index of this structlog in this transaction' CODEC(DoubleDelta, ZSTD(1)), + `operation` LowCardinality(String) COMMENT 'The operation', + `gas` UInt64 COMMENT 'The gas' CODEC(Delta(8), ZSTD(1)), + `gas_cost` UInt64 COMMENT 'The gas cost' CODEC(DoubleDelta, ZSTD(1)), + `gas_used` UInt64 DEFAULT 0 COMMENT 'Actual gas consumed (computed from consecutive gas values)' CODEC(ZSTD(1)), + `gas_self` UInt64 DEFAULT 0 COMMENT 'Gas consumed by this opcode only, excludes child frame gas for CALL/CREATE opcodes. sum(gas_self) = total execution gas without double counting' CODEC(ZSTD(1)), + `depth` UInt64 COMMENT 'The depth' CODEC(DoubleDelta, ZSTD(1)), + `return_data` Nullable(String) COMMENT 'The return data' CODEC(ZSTD(1)), + `refund` Nullable(UInt64) COMMENT 'The refund' CODEC(ZSTD(1)), + `error` Nullable(String) COMMENT 'The error' CODEC(ZSTD(1)), + `call_to_address` Nullable(String) COMMENT 'Address of a CALL operation' CODEC(ZSTD(1)), + `call_frame_id` UInt32 DEFAULT 0 COMMENT 'Sequential identifier for the call frame within the transaction' CODEC(DoubleDelta, ZSTD(1)), + `call_frame_path` Array(UInt32) DEFAULT [0] COMMENT 'Path of frame IDs from root to current frame' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 201600)) +ORDER BY (meta_network_name, block_number, transaction_hash, index) +COMMENT 'Contains canonical execution transaction structlog data.'; + +CREATE TABLE IF NOT EXISTS default.consensus_engine_api_get_blobs_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event' CODEC(DoubleDelta, ZSTD(1)), + `requested_date_time` DateTime64(3) COMMENT 'Timestamp when the engine_getBlobs call was initiated' CODEC(DoubleDelta, ZSTD(1)), + `duration_ms` UInt64 COMMENT 'How long the engine_getBlobs call took in milliseconds' CODEC(ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number of the beacon block being reconstructed' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number derived from the slot' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'Root of the beacon block (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `parent_block_root` FixedString(66) COMMENT 'Root of the parent beacon block (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `requested_count` UInt32 COMMENT 'Number of versioned hashes requested' CODEC(ZSTD(1)), + `versioned_hashes` Array(FixedString(66)) COMMENT 'List of versioned hashes requested (derived from KZG commitments)' CODEC(ZSTD(1)), + `returned_count` UInt32 COMMENT 'Number of non-null blobs returned' CODEC(ZSTD(1)), + `status` LowCardinality(String) COMMENT 'Result status (SUCCESS, PARTIAL, EMPTY, UNSUPPORTED, ERROR)', + `error_message` Nullable(String) COMMENT 'Error details if status is ERROR or UNSUPPORTED' CODEC(ZSTD(1)), + `method_version` LowCardinality(String) COMMENT 'Version of the engine_getBlobs method (e.g., V1, V2)', + `meta_execution_version` LowCardinality(String) COMMENT 'Full execution client version string from web3_clientVersion RPC', + `meta_execution_implementation` LowCardinality(String) COMMENT 'Execution client implementation name (e.g., Geth, Nethermind, Besu, Reth, Erigon)', + `meta_execution_version_major` LowCardinality(String) COMMENT 'Execution client major version number', + `meta_execution_version_minor` LowCardinality(String) COMMENT 'Execution client minor version number', + `meta_execution_version_patch` LowCardinality(String) COMMENT 'Execution client patch version number', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block_root, event_date_time) +COMMENT 'Contains timing and instrumentation data for engine_getBlobs calls between the consensus and execution layer.'; + +CREATE TABLE IF NOT EXISTS default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event' CODEC(DoubleDelta, ZSTD(1)), + `requested_date_time` DateTime64(3) COMMENT 'Timestamp when the engine_newPayload call was initiated' CODEC(DoubleDelta, ZSTD(1)), + `duration_ms` UInt64 COMMENT 'How long the engine_newPayload call took in milliseconds' CODEC(ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number of the beacon block containing the payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number derived from the slot' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `block_root` FixedString(66) COMMENT 'Root of the beacon block (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `parent_block_root` FixedString(66) COMMENT 'Root of the parent beacon block (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `proposer_index` UInt32 COMMENT 'Validator index of the block proposer' CODEC(ZSTD(1)), + `block_number` UInt64 COMMENT 'Execution block number' CODEC(DoubleDelta, ZSTD(1)), + `block_hash` FixedString(66) COMMENT 'Execution block hash (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `parent_hash` FixedString(66) COMMENT 'Parent execution block hash (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `gas_used` UInt64 COMMENT 'Total gas used by all transactions in the block' CODEC(ZSTD(1)), + `gas_limit` UInt64 COMMENT 'Gas limit of the block' CODEC(ZSTD(1)), + `tx_count` UInt32 COMMENT 'Number of transactions in the block' CODEC(ZSTD(1)), + `blob_count` UInt32 COMMENT 'Number of blobs in the block' CODEC(ZSTD(1)), + `status` LowCardinality(String) COMMENT 'Payload status returned by EL (VALID, INVALID, SYNCING, ACCEPTED, INVALID_BLOCK_HASH)', + `latest_valid_hash` Nullable(FixedString(66)) COMMENT 'Latest valid hash when status is INVALID (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `validation_error` Nullable(String) COMMENT 'Error message when validation fails' CODEC(ZSTD(1)), + `method_version` LowCardinality(String) COMMENT 'Version of the engine_newPayload method (e.g., V3, V4)', + `meta_execution_version` LowCardinality(String), + `meta_execution_implementation` LowCardinality(String) DEFAULT '' COMMENT 'Execution client implementation name (e.g., Geth, Nethermind, Besu, Reth, Erigon)', + `meta_execution_version_major` LowCardinality(String) DEFAULT '' COMMENT 'Execution client major version number', + `meta_execution_version_minor` LowCardinality(String) DEFAULT '' COMMENT 'Execution client minor version number', + `meta_execution_version_patch` LowCardinality(String) DEFAULT '' COMMENT 'Execution client patch version number', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, block_hash, event_date_time) +COMMENT 'Contains timing and instrumentation data for engine_newPayload calls between the consensus and execution layer.'; + +CREATE TABLE IF NOT EXISTS default.ethseer_validator_entity_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from ethseer.io' CODEC(DoubleDelta, ZSTD(1)), + `index` UInt32 COMMENT 'The index of the validator' CODEC(DoubleDelta, ZSTD(1)), + `pubkey` String COMMENT 'The public key of the validator' CODEC(ZSTD(1)), + `entity` String COMMENT 'The entity of the validator' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY meta_network_name +ORDER BY (meta_network_name, index, pubkey) +COMMENT 'Contains a mapping of validators to entities'; + +CREATE TABLE IF NOT EXISTS default.execution_block_metrics_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the event was received' CODEC(DoubleDelta, ZSTD(1)), + `source` LowCardinality(String) COMMENT 'Data source (e.g., client-logs)', + `block_number` UInt64 COMMENT 'Execution block number' CODEC(DoubleDelta, ZSTD(1)), + `block_hash` FixedString(66) COMMENT 'Execution block hash (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `gas_used` UInt64 COMMENT 'Total gas used by all transactions in the block' CODEC(ZSTD(1)), + `tx_count` UInt32 COMMENT 'Number of transactions in the block' CODEC(ZSTD(1)), + `execution_ms` Float64 COMMENT 'Time spent executing transactions in milliseconds' CODEC(ZSTD(1)), + `state_read_ms` Float64 COMMENT 'Time spent reading state in milliseconds' CODEC(ZSTD(1)), + `state_hash_ms` Float64 COMMENT 'Time spent computing state hash in milliseconds' CODEC(ZSTD(1)), + `commit_ms` Float64 COMMENT 'Time spent committing state changes in milliseconds' CODEC(ZSTD(1)), + `total_ms` Float64 COMMENT 'Total time for block processing in milliseconds' CODEC(ZSTD(1)), + `mgas_per_sec` Float64 COMMENT 'Throughput in million gas per second' CODEC(ZSTD(1)), + `state_reads_accounts` UInt64 COMMENT 'Number of account reads' CODEC(ZSTD(1)), + `state_reads_storage_slots` UInt64 COMMENT 'Number of storage slot reads' CODEC(ZSTD(1)), + `state_reads_code` UInt64 COMMENT 'Number of code reads' CODEC(ZSTD(1)), + `state_reads_code_bytes` UInt64 COMMENT 'Total bytes of code read' CODEC(ZSTD(1)), + `state_writes_accounts` UInt64 COMMENT 'Number of account writes' CODEC(ZSTD(1)), + `state_writes_accounts_deleted` UInt64 COMMENT 'Number of accounts deleted' CODEC(ZSTD(1)), + `state_writes_storage_slots` UInt64 COMMENT 'Number of storage slot writes' CODEC(ZSTD(1)), + `state_writes_storage_slots_deleted` UInt64 COMMENT 'Number of storage slots deleted' CODEC(ZSTD(1)), + `state_writes_code` UInt64 COMMENT 'Number of code writes' CODEC(ZSTD(1)), + `state_writes_code_bytes` UInt64 COMMENT 'Total bytes of code written' CODEC(ZSTD(1)), + `account_cache_hits` Int64 COMMENT 'Number of account cache hits' CODEC(ZSTD(1)), + `account_cache_misses` Int64 COMMENT 'Number of account cache misses' CODEC(ZSTD(1)), + `account_cache_hit_rate` Float64 COMMENT 'Account cache hit rate as percentage' CODEC(ZSTD(1)), + `storage_cache_hits` Int64 COMMENT 'Number of storage cache hits' CODEC(ZSTD(1)), + `storage_cache_misses` Int64 COMMENT 'Number of storage cache misses' CODEC(ZSTD(1)), + `storage_cache_hit_rate` Float64 COMMENT 'Storage cache hit rate as percentage' CODEC(ZSTD(1)), + `code_cache_hits` Int64 COMMENT 'Number of code cache hits' CODEC(ZSTD(1)), + `code_cache_misses` Int64 COMMENT 'Number of code cache misses' CODEC(ZSTD(1)), + `code_cache_hit_rate` Float64 COMMENT 'Code cache hit rate as percentage' CODEC(ZSTD(1)), + `code_cache_hit_bytes` Int64 COMMENT 'Total bytes of code cache hits' CODEC(ZSTD(1)), + `code_cache_miss_bytes` Int64 COMMENT 'Total bytes of code cache misses' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, meta_client_name, event_date_time) +COMMENT 'Contains detailed performance metrics from execution client structured logging for block execution'; + +CREATE TABLE IF NOT EXISTS default.execution_engine_get_blobs_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the event was received' CODEC(DoubleDelta, ZSTD(1)), + `requested_date_time` DateTime64(3) COMMENT 'Timestamp when the engine_getBlobs call was received' CODEC(DoubleDelta, ZSTD(1)), + `duration_ms` UInt64 COMMENT 'How long the engine_getBlobs call took in milliseconds' CODEC(ZSTD(1)), + `source` LowCardinality(String) COMMENT 'Source of the event (SNOOPER, EXECUTION_CLIENT)', + `requested_count` UInt32 COMMENT 'Number of versioned hashes requested' CODEC(ZSTD(1)), + `versioned_hashes` Array(FixedString(66)) COMMENT 'List of versioned hashes requested (hex encoded)' CODEC(ZSTD(1)), + `returned_count` UInt32 COMMENT 'Number of non-null blobs returned' CODEC(ZSTD(1)), + `returned_blob_indexes` Array(UInt8) COMMENT 'Indexes (0-based) of the requested versioned_hashes that were successfully returned', + `status` LowCardinality(String) COMMENT 'Result status (SUCCESS, PARTIAL, EMPTY, UNSUPPORTED, ERROR)', + `error_message` Nullable(String) COMMENT 'Error details if status is ERROR or UNSUPPORTED' CODEC(ZSTD(1)), + `method_version` LowCardinality(String) COMMENT 'Version of the engine_getBlobs method (e.g., V1, V2)', + `meta_execution_implementation` LowCardinality(String) COMMENT 'Implementation of the execution client (e.g., go-ethereum, reth, nethermind)', + `meta_execution_version` LowCardinality(String) COMMENT 'Version of the execution client', + `meta_execution_version_major` LowCardinality(String) COMMENT 'Major version number of the execution client', + `meta_execution_version_minor` LowCardinality(String) COMMENT 'Minor version number of the execution client', + `meta_execution_version_patch` LowCardinality(String) COMMENT 'Patch version number of the execution client', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name) +COMMENT 'Contains timing and instrumentation data for engine_getBlobs calls from the execution layer perspective.'; + +CREATE TABLE IF NOT EXISTS default.execution_engine_new_payload_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the event was received' CODEC(DoubleDelta, ZSTD(1)), + `requested_date_time` DateTime64(3) COMMENT 'Timestamp when the engine_newPayload call was received' CODEC(DoubleDelta, ZSTD(1)), + `duration_ms` UInt64 COMMENT 'How long the engine_newPayload call took in milliseconds' CODEC(ZSTD(1)), + `source` LowCardinality(String) COMMENT 'Source of the event (SNOOPER, EXECUTION_CLIENT)', + `block_number` UInt64 COMMENT 'Execution block number' CODEC(DoubleDelta, ZSTD(1)), + `block_hash` FixedString(66) COMMENT 'Execution block hash (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `parent_hash` FixedString(66) COMMENT 'Parent execution block hash (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `gas_used` UInt64 COMMENT 'Total gas used by all transactions in the block' CODEC(ZSTD(1)), + `gas_limit` UInt64 COMMENT 'Gas limit of the block' CODEC(ZSTD(1)), + `tx_count` UInt32 COMMENT 'Number of transactions in the block' CODEC(ZSTD(1)), + `blob_count` UInt32 COMMENT 'Number of blobs in the block' CODEC(ZSTD(1)), + `status` LowCardinality(String) COMMENT 'Payload status returned (VALID, INVALID, SYNCING, ACCEPTED, INVALID_BLOCK_HASH)', + `latest_valid_hash` Nullable(FixedString(66)) COMMENT 'Latest valid hash when status is INVALID (hex encoded with 0x prefix)' CODEC(ZSTD(1)), + `validation_error` Nullable(String) COMMENT 'Error message when validation fails' CODEC(ZSTD(1)), + `method_version` LowCardinality(String) COMMENT 'Version of the engine_newPayload method (e.g., V3, V4)', + `meta_execution_implementation` LowCardinality(String) COMMENT 'Implementation of the execution client (e.g., go-ethereum, reth, nethermind)', + `meta_execution_version` LowCardinality(String) COMMENT 'Version of the execution client', + `meta_execution_version_major` LowCardinality(String) COMMENT 'Major version number of the execution client', + `meta_execution_version_minor` LowCardinality(String) COMMENT 'Minor version number of the execution client', + `meta_execution_version_patch` LowCardinality(String) COMMENT 'Patch version number of the execution client', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, meta_client_name, block_hash, event_date_time) +COMMENT 'Contains timing and instrumentation data for engine_newPayload calls from the execution layer perspective.'; + +CREATE TABLE IF NOT EXISTS default.execution_state_size_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the state size measurement was taken' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'Block number at which the state size was measured' CODEC(DoubleDelta, ZSTD(1)), + `state_root` FixedString(66) COMMENT 'State root hash of the execution layer at this block' CODEC(ZSTD(1)), + `accounts` UInt64 COMMENT 'Total number of accounts in the state' CODEC(ZSTD(1)), + `account_bytes` UInt64 COMMENT 'Total bytes used by account data' CODEC(ZSTD(1)), + `account_trienodes` UInt64 COMMENT 'Number of trie nodes in the account trie' CODEC(ZSTD(1)), + `account_trienode_bytes` UInt64 COMMENT 'Total bytes used by account trie nodes' CODEC(ZSTD(1)), + `contract_codes` UInt64 COMMENT 'Total number of contract codes stored' CODEC(ZSTD(1)), + `contract_code_bytes` UInt64 COMMENT 'Total bytes used by contract code' CODEC(ZSTD(1)), + `storages` UInt64 COMMENT 'Total number of storage slots in the state' CODEC(ZSTD(1)), + `storage_bytes` UInt64 COMMENT 'Total bytes used by storage data' CODEC(ZSTD(1)), + `storage_trienodes` UInt64 COMMENT 'Number of trie nodes in the storage trie' CODEC(ZSTD(1)), + `storage_trienode_bytes` UInt64 COMMENT 'Total bytes used by storage trie nodes' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_execution_version` LowCardinality(String) COMMENT 'Execution client version that generated the event', + `meta_execution_version_major` LowCardinality(String) COMMENT 'Execution client major version that generated the event', + `meta_execution_version_minor` LowCardinality(String) COMMENT 'Execution client minor version that generated the event', + `meta_execution_version_patch` LowCardinality(String) COMMENT 'Execution client patch version that generated the event', + `meta_execution_implementation` LowCardinality(String) COMMENT 'Execution client implementation that generated the event' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, meta_client_name, state_root, event_date_time) +COMMENT 'Contains execution layer state size metrics including account, contract code, and storage data measurements at specific block heights.'; + +CREATE TABLE IF NOT EXISTS default.execution_transaction_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `block_hash` FixedString(66) COMMENT 'The block hash' CODEC(ZSTD(1)), + `parent_hash` FixedString(66) COMMENT 'The parent block hash' CODEC(ZSTD(1)), + `position` UInt32 COMMENT 'The position of the transaction in the beacon block' CODEC(DoubleDelta, ZSTD(1)), + `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), + `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), + `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), + `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), + `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), + `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), + `gas_tip_cap` Nullable(UInt128) COMMENT 'The priority fee (tip) the user has set for the transaction' CODEC(ZSTD(1)), + `gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), + `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), + `type` UInt8 COMMENT 'The type of the transaction' CODEC(ZSTD(1)), + `size` UInt32 COMMENT 'The size of the transaction data in bytes' CODEC(ZSTD(1)), + `call_data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), + `blob_gas` Nullable(UInt64) COMMENT 'The maximum gas provided for the blob transaction execution' CODEC(ZSTD(1)), + `blob_gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), + `blob_hashes` Array(String) COMMENT 'The hashes of the blob commitments for blob transactions' CODEC(ZSTD(1)), + `success` Bool COMMENT 'The transaction success' CODEC(ZSTD(1)), + `n_input_bytes` UInt32 COMMENT 'The transaction input bytes' CODEC(ZSTD(1)), + `n_input_zero_bytes` UInt32 COMMENT 'The transaction input zero bytes' CODEC(ZSTD(1)), + `n_input_nonzero_bytes` UInt32 COMMENT 'The transaction input nonzero bytes' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, intDiv(block_number, 5000000)) +ORDER BY (meta_network_name, block_number, block_hash, position) +COMMENT 'Contains execution transaction data that may not be canonical.'; + +CREATE TABLE IF NOT EXISTS default.imported_sources_local ON CLUSTER '{cluster}' +( + `create_date_time` DateTime64(3) COMMENT 'Creation date of this row' CODEC(DoubleDelta, ZSTD(1)), + `target_date_time` DateTime COMMENT 'The date of the data that was imported' CODEC(DoubleDelta, ZSTD(1)), + `source` LowCardinality(String) COMMENT 'Source of the data that was imported' +) +ENGINE = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') +PARTITION BY toStartOfMonth(create_date_time) +ORDER BY (create_date_time, source) +COMMENT 'This table contains the list of sources that have been imported into the database'; + +CREATE TABLE IF NOT EXISTS default.libp2p_add_peer_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `protocol` LowCardinality(String) COMMENT 'Protocol used by the peer', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key) +COMMENT 'Contains the details of the peers added to the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_connected_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `remote_peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the remote peer', + `remote_protocol` LowCardinality(String) COMMENT 'Protocol of the remote peer', + `remote_transport_protocol` LowCardinality(String) COMMENT 'Transport protocol of the remote peer', + `remote_port` Nullable(UInt16) COMMENT 'Port of the remote peer' CODEC(ZSTD(1)), + `remote_ip` Nullable(IPv6) COMMENT 'IP address of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_city` LowCardinality(String) COMMENT 'City of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_country` LowCardinality(String) COMMENT 'Country of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_country_code` LowCardinality(String) COMMENT 'Country code of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_agent_implementation` LowCardinality(String) COMMENT 'Implementation of the remote peer', + `remote_agent_version` LowCardinality(String) COMMENT 'Version of the remote peer', + `remote_agent_version_major` LowCardinality(String) COMMENT 'Major version of the remote peer', + `remote_agent_version_minor` LowCardinality(String) COMMENT 'Minor version of the remote peer', + `remote_agent_version_patch` LowCardinality(String) COMMENT 'Patch version of the remote peer', + `remote_agent_platform` LowCardinality(String) COMMENT 'Platform of the remote peer', + `direction` LowCardinality(String) COMMENT 'Connection direction', + `opened` DateTime COMMENT 'Timestamp when the connection was opened' CODEC(DoubleDelta, ZSTD(1)), + `transient` Bool COMMENT 'Whether the connection is transient', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, remote_peer_id_unique_key, direction, opened) +COMMENT 'Contains the details of the CONNECTED events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_deliver_message_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `seq_number` UInt64 COMMENT 'A linearly increasing number that is unique among messages originating from the given peer' CODEC(DoubleDelta, ZSTD(1)), + `local_delivery` Bool COMMENT 'Indicates if the message was delivered to in-process subscribers only', + `peer_id_unique_key` Int64 COMMENT 'Unique key for the peer that delivered the message', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name, message_id, seq_number) +COMMENT 'Contains the details of the DELIVER_MESSAGE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_disconnected_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `remote_peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the remote peer', + `remote_protocol` LowCardinality(String) COMMENT 'Protocol of the remote peer', + `remote_transport_protocol` LowCardinality(String) COMMENT 'Transport protocol of the remote peer', + `remote_port` Nullable(UInt16) COMMENT 'Port of the remote peer' CODEC(ZSTD(1)), + `remote_ip` Nullable(IPv6) COMMENT 'IP address of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_city` LowCardinality(String) COMMENT 'City of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_country` LowCardinality(String) COMMENT 'Country of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_country_code` LowCardinality(String) COMMENT 'Country code of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the remote peer that generated the event' CODEC(ZSTD(1)), + `remote_agent_implementation` LowCardinality(String) COMMENT 'Implementation of the remote peer', + `remote_agent_version` LowCardinality(String) COMMENT 'Version of the remote peer', + `remote_agent_version_major` LowCardinality(String) COMMENT 'Major version of the remote peer', + `remote_agent_version_minor` LowCardinality(String) COMMENT 'Minor version of the remote peer', + `remote_agent_version_patch` LowCardinality(String) COMMENT 'Patch version of the remote peer', + `remote_agent_platform` LowCardinality(String) COMMENT 'Platform of the remote peer', + `direction` LowCardinality(String) COMMENT 'Connection direction', + `opened` DateTime COMMENT 'Timestamp when the connection was opened' CODEC(DoubleDelta, ZSTD(1)), + `transient` Bool COMMENT 'Whether the connection is transient', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, remote_peer_id_unique_key, direction, opened) +COMMENT 'Contains the details of the DISCONNECTED events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_drop_rpc_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each record', + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer receiver', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, meta_client_name) +COMMENT 'Contains the details of the RPC messages dropped by the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_duplicate_message_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `seq_number` UInt64 COMMENT 'A linearly increasing number that is unique among messages originating from the given peer' CODEC(DoubleDelta, ZSTD(1)), + `local_delivery` Bool COMMENT 'Indicates if the message was duplicated locally', + `peer_id_unique_key` Int64 COMMENT 'Unique key for the peer that sent the duplicate message', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name, message_id, seq_number) +COMMENT 'Contains the details of the DUPLICATE_MESSAGE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_aggregate_and_proof_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', + `aggregator_index` UInt32 COMMENT 'Index of the validator who created this aggregate' CODEC(DoubleDelta, ZSTD(1)), + `committee_index` LowCardinality(String) COMMENT 'Committee index from the attestation', + `aggregation_bits` String COMMENT 'Bitfield of aggregated attestation' CODEC(ZSTD(1)), + `beacon_block_root` FixedString(66) COMMENT 'Root of the beacon block being attested to' CODEC(ZSTD(1)), + `source_epoch` UInt32 COMMENT 'Source epoch from the attestation' CODEC(DoubleDelta, ZSTD(1)), + `source_root` FixedString(66) COMMENT 'Source root from the attestation' CODEC(ZSTD(1)), + `target_epoch` UInt32 COMMENT 'Target epoch from the attestation' CODEC(DoubleDelta, ZSTD(1)), + `target_root` FixedString(66) COMMENT 'Target root from the attestation' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, peer_id_unique_key, message_id) +COMMENT 'Table for libp2p gossipsub aggregate and proof data.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'The epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `committee_index` LowCardinality(String) COMMENT 'The committee index in the attestation', + `attesting_validator_index` Nullable(UInt32) COMMENT 'The index of the validator attesting to the event' CODEC(ZSTD(1)), + `attesting_validator_committee_index` LowCardinality(String) COMMENT 'The committee index of the attesting validator', + `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', + `aggregation_bits` String COMMENT 'The aggregation bits of the event in the attestation' CODEC(ZSTD(1)), + `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the attestation' CODEC(ZSTD(1)), + `source_epoch` UInt32 COMMENT 'The source epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), + `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), + `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the attestation' CODEC(ZSTD(1)), + `target_epoch` UInt32 COMMENT 'The target epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), + `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), + `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the attestation' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, peer_id_unique_key, message_id) +COMMENT 'Table for libp2p gossipsub beacon attestation data.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), + `block` FixedString(66) COMMENT 'The beacon block root hash' CODEC(ZSTD(1)), + `proposer_index` UInt32 COMMENT 'The proposer index of the beacon block' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, peer_id_unique_key, message_id) +COMMENT 'Table for libp2p gossipsub beacon block data.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), + `proposer_index` UInt32 COMMENT 'The proposer index of the beacon block' CODEC(ZSTD(1)), + `blob_index` UInt32 COMMENT 'Blob index associated with the record' CODEC(ZSTD(1)), + `beacon_block_root` FixedString(66) CODEC(ZSTD(1)), + `parent_root` FixedString(66) COMMENT 'Parent root of the beacon block' CODEC(ZSTD(1)), + `state_root` FixedString(66) COMMENT 'State root of the beacon block' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, peer_id_unique_key, message_id) +COMMENT 'Table for libp2p gossipsub blob sidecar data'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_data_column_sidecar_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), + `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), + `proposer_index` UInt32 COMMENT 'The proposer index of the beacon block' CODEC(ZSTD(1)), + `column_index` UInt64 COMMENT 'Column index associated with the record' CODEC(ZSTD(1)), + `kzg_commitments_count` UInt32 COMMENT 'Number of KZG commitments associated with the record' CODEC(ZSTD(1)), + `beacon_block_root` FixedString(66) CODEC(ZSTD(1)), + `parent_root` FixedString(66) COMMENT 'Parent root of the beacon block' CODEC(ZSTD(1)), + `state_root` FixedString(66) COMMENT 'State root of the beacon block' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, peer_id_unique_key, message_id) +COMMENT 'Table for libp2p gossipsub data column sidecar data'; + +CREATE TABLE IF NOT EXISTS default.libp2p_graft_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `peer_id_unique_key` Int64 COMMENT 'Unique key for the peer that initiated the GRAFT (eg joined the mesh for this topic) identifies mesh membership changes per peer.', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name) +COMMENT 'Contains the details of the GRAFT events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_handle_metadata_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the RPC', + `error` Nullable(String) COMMENT 'Error message if the metadata handling failed' CODEC(ZSTD(1)), + `protocol` LowCardinality(String) COMMENT 'The protocol of the metadata handling event', + `direction` LowCardinality(String) COMMENT 'Direction of the RPC request (inbound or outbound)' CODEC(ZSTD(1)), + `attnets` String COMMENT 'Attestation subnets the peer is subscribed to' CODEC(ZSTD(1)), + `seq_number` UInt64 COMMENT 'Sequence number of the metadata' CODEC(DoubleDelta, ZSTD(1)), + `syncnets` String COMMENT 'Sync subnets the peer is subscribed to' CODEC(ZSTD(1)), + `custody_group_count` Nullable(UInt8) COMMENT 'Number of custody groups (0-127)' CODEC(ZSTD(1)), + `latency_milliseconds` Decimal(10, 3) COMMENT 'How long it took to handle the metadata request in milliseconds' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, attnets, seq_number, syncnets, latency_milliseconds) +COMMENT 'Contains the metadata handling events for libp2p peers.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_handle_status_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `error` Nullable(String) COMMENT 'Error message if the status handling failed' CODEC(ZSTD(1)), + `protocol` LowCardinality(String) COMMENT 'The protocol of the status handling event', + `direction` LowCardinality(String) COMMENT 'Direction of the RPC request (inbound or outbound)' CODEC(ZSTD(1)), + `request_finalized_epoch` Nullable(UInt32) COMMENT 'Requested finalized epoch' CODEC(DoubleDelta, ZSTD(1)), + `request_finalized_root` Nullable(String) COMMENT 'Requested finalized root', + `request_fork_digest` LowCardinality(String) COMMENT 'Requested fork digest', + `request_head_root` Nullable(FixedString(66)) COMMENT 'Requested head root' CODEC(ZSTD(1)), + `request_head_slot` Nullable(UInt32) COMMENT 'Requested head slot' CODEC(ZSTD(1)), + `request_earliest_available_slot` Nullable(UInt32) COMMENT 'Requested earliest available slot' CODEC(ZSTD(1)), + `response_finalized_epoch` Nullable(UInt32) COMMENT 'Response finalized epoch' CODEC(DoubleDelta, ZSTD(1)), + `response_finalized_root` Nullable(FixedString(66)) COMMENT 'Response finalized root' CODEC(ZSTD(1)), + `response_fork_digest` LowCardinality(String) COMMENT 'Response fork digest', + `response_head_root` Nullable(FixedString(66)) COMMENT 'Response head root' CODEC(ZSTD(1)), + `response_head_slot` Nullable(UInt32) COMMENT 'Response head slot' CODEC(DoubleDelta, ZSTD(1)), + `response_earliest_available_slot` Nullable(UInt32) COMMENT 'Response earliest available slot' CODEC(ZSTD(1)), + `latency_milliseconds` Decimal(10, 3) COMMENT 'How long it took to handle the status request in milliseconds' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, latency_milliseconds) +COMMENT 'Contains the status handling events for libp2p peers.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_identify_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `remote_peer_id_unique_key` Int64 CODEC(ZSTD(1)), + `success` Bool CODEC(ZSTD(1)), + `error` Nullable(String) CODEC(ZSTD(1)), + `remote_protocol` LowCardinality(String), + `remote_transport_protocol` LowCardinality(String), + `remote_port` UInt16 CODEC(ZSTD(1)), + `remote_ip` Nullable(IPv6) CODEC(ZSTD(1)), + `remote_geo_city` LowCardinality(String) CODEC(ZSTD(1)), + `remote_geo_country` LowCardinality(String) CODEC(ZSTD(1)), + `remote_geo_country_code` LowCardinality(String) CODEC(ZSTD(1)), + `remote_geo_continent_code` LowCardinality(String) CODEC(ZSTD(1)), + `remote_geo_longitude` Nullable(Float64) CODEC(ZSTD(1)), + `remote_geo_latitude` Nullable(Float64) CODEC(ZSTD(1)), + `remote_geo_autonomous_system_number` Nullable(UInt32) CODEC(ZSTD(1)), + `remote_geo_autonomous_system_organization` Nullable(String) CODEC(ZSTD(1)), + `remote_agent_implementation` LowCardinality(String), + `remote_agent_version` LowCardinality(String), + `remote_agent_version_major` LowCardinality(String), + `remote_agent_version_minor` LowCardinality(String), + `remote_agent_version_patch` LowCardinality(String), + `remote_agent_platform` LowCardinality(String), + `protocol_version` LowCardinality(String), + `protocols` Array(String) CODEC(ZSTD(1)), + `listen_addrs` Array(String) CODEC(ZSTD(1)), + `observed_addr` String CODEC(ZSTD(1)), + `transport` LowCardinality(String), + `security` LowCardinality(String), + `muxer` LowCardinality(String), + `direction` LowCardinality(String), + `remote_multiaddr` String CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_client_version` LowCardinality(String), + `meta_client_implementation` LowCardinality(String), + `meta_client_os` LowCardinality(String), + `meta_client_ip` Nullable(IPv6) CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, remote_peer_id_unique_key, direction) +COMMENT 'Contains libp2p identify protocol exchange results including remote peer agent info, supported protocols, and connection metadata'; + +CREATE TABLE IF NOT EXISTS default.libp2p_join_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer that joined the topic', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name) +COMMENT 'Contains the details of the JOIN events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_leave_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer that left the topic', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name) +COMMENT 'Contains the details of the LEAVE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_peer_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each record, seahash of peer_id + meta_network_name', + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `peer_id` String COMMENT 'Peer ID' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY meta_network_name +ORDER BY unique_key +COMMENT 'Contains the original peer id of a seahashed peer_id + meta_network_name, commonly seen in other tables as the field peer_id_unique_key'; + +CREATE TABLE IF NOT EXISTS default.libp2p_prune_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `peer_id_unique_key` Int64 COMMENT 'Unique key for the peer that was PRUNED (eg removed from the mesh for this topic) identifies mesh membership changes per peer.', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name) +COMMENT 'Contains the details of the PRUNE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_publish_message_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, topic_fork_digest_value, topic_name, message_id) +COMMENT 'Contains the details of the PUBLISH_MESSAGE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_recv_rpc_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each record', + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer sender', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, meta_client_name) +COMMENT 'Contains the details of the RPC messages received by the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_reject_message_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `seq_number` UInt64 COMMENT 'A linearly increasing number that is unique among messages originating from the given peer' CODEC(DoubleDelta, ZSTD(1)), + `local_delivery` Bool COMMENT 'Indicates if the message was rejected by local subscriber', + `peer_id_unique_key` Int64 COMMENT 'Unique key for the peer that rejected the message', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), + `reason` String COMMENT 'Reason for message rejection' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name, message_id, seq_number) +COMMENT 'Contains the details of the REJECT_MESSAGE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_remove_peer_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key) +COMMENT 'Contains the details of the peers removed from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_data_column_custody_probe_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the probe was executed' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number being probed' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number of the slot being probed' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_request_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_request_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_request_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_request_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `column_index` UInt64 COMMENT 'Column index being probed' CODEC(ZSTD(1)), + `column_rows_count` UInt16 COMMENT 'Number of rows in the column' CODEC(ZSTD(1)), + `beacon_block_root` FixedString(66) COMMENT 'Root of the beacon block' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', + `result` LowCardinality(String) COMMENT 'Result of the probe' CODEC(ZSTD(1)), + `response_time_ms` Int32 COMMENT 'Response time in milliseconds' CODEC(ZSTD(1)), + `error` Nullable(String) COMMENT 'Error message if probe failed' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that executed the probe', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, peer_id_unique_key, slot, column_index) +COMMENT 'Contains custody probe events for data column availability verification'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_graft_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each "Graft" control record', + `updated_date_time` DateTime COMMENT 'Timestamp when the "Graft" control record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the "Graft" control event' CODEC(DoubleDelta, ZSTD(1)), + `control_index` Int32 COMMENT 'Position in the RPC meta control GRAFT array' CODEC(DoubleDelta, ZSTD(1)), + `rpc_meta_unique_key` Int64 COMMENT 'Unique key associated with the "Graft" control metadata', + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the Graft control', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, control_index, meta_client_name) +COMMENT 'Contains the details of the "Graft" control messages from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_idontwant_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each IDONTWANT control record', + `updated_date_time` DateTime COMMENT 'Timestamp when the IDONTWANT control record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the IDONTWANT control event' CODEC(DoubleDelta, ZSTD(1)), + `control_index` Int32 COMMENT 'Position in the RPC meta control idontwant array' CODEC(DoubleDelta, ZSTD(1)), + `message_index` Int32 COMMENT 'Position in the RPC meta control idontwant message_ids array' CODEC(DoubleDelta, ZSTD(1)), + `rpc_meta_unique_key` Int64 COMMENT 'Unique key associated with the IDONTWANT control metadata', + `message_id` String COMMENT 'Identifier of the message associated with the IDONTWANT control' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the IDONTWANT control', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, control_index, peer_id_unique_key, message_id, message_index, meta_client_name) +COMMENT 'Contains the details of the IDONTWANT control messages from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_ihave_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each "I have" control record', + `updated_date_time` DateTime COMMENT 'Timestamp when the "I have" control record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the "I have" control event' CODEC(DoubleDelta, ZSTD(1)), + `rpc_meta_unique_key` Int64 COMMENT 'Unique key associated with the "I have" control metadata', + `message_index` Int32 COMMENT 'Position in the RPC meta control IWANT message_ids array' CODEC(DoubleDelta, ZSTD(1)), + `control_index` Int32 COMMENT 'Position in the RPC meta control IWANT array' CODEC(DoubleDelta, ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `message_id` String COMMENT 'Identifier of the message associated with the "I have" control' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the I have control', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, control_index, message_index, meta_client_name) +COMMENT 'Contains the details of the "I have" control messages from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_iwant_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each "I want" control record', + `updated_date_time` DateTime COMMENT 'Timestamp when the "I want" control record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the "I want" control event' CODEC(DoubleDelta, ZSTD(1)), + `control_index` Int32 COMMENT 'Position in the RPC meta control IWANT array' CODEC(DoubleDelta, ZSTD(1)), + `message_index` Int32 COMMENT 'Position in the RPC meta control IWANT message_ids array' CODEC(DoubleDelta, ZSTD(1)), + `rpc_meta_unique_key` Int64 COMMENT 'Unique key associated with the "I want" control metadata', + `message_id` String COMMENT 'Identifier of the message associated with the "I want" control' CODEC(ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the I want control', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, control_index, message_index, meta_client_name) +COMMENT 'Contains IWANT control messages from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Peers request specific message IDs. Partition: monthly by `event_date_time`'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_prune_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each "Prune" control record', + `updated_date_time` DateTime COMMENT 'Timestamp when the "Prune" control record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the "Prune" control event' CODEC(DoubleDelta, ZSTD(1)), + `control_index` Int32 COMMENT 'Position in the RPC meta control PRUNE array' CODEC(DoubleDelta, ZSTD(1)), + `rpc_meta_unique_key` Int64 COMMENT 'Unique key associated with the "Prune" control metadata', + `peer_id_index` Int32 CODEC(DoubleDelta, ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the Prune control', + `graft_peer_id_unique_key` Nullable(Int64) COMMENT 'Unique key associated with the identifier of the graft peer involved in the Prune control', + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, control_index, meta_client_name) +COMMENT 'Contains the details of the "Prune" control messages from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_message_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each RPC message record', + `updated_date_time` DateTime COMMENT 'Timestamp when the RPC message record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the RPC event' CODEC(DoubleDelta, ZSTD(1)), + `control_index` Int32 COMMENT 'Position in the RPC meta message array' CODEC(DoubleDelta, ZSTD(1)), + `rpc_meta_unique_key` Int64 COMMENT 'Unique key associated with the RPC metadata', + `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the RPC', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, control_index, meta_client_name) +COMMENT 'Contains the details of the RPC meta messages from the peer'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_subscription_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each RPC subscription record', + `updated_date_time` DateTime COMMENT 'Timestamp when the RPC subscription record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the RPC subscription event' CODEC(DoubleDelta, ZSTD(1)), + `control_index` Int32 COMMENT 'Position in the RPC meta subscription array' CODEC(DoubleDelta, ZSTD(1)), + `rpc_meta_unique_key` Int64 COMMENT 'Unique key associated with the RPC subscription metadata', + `subscribe` Bool COMMENT 'Boolean indicating if it is a subscription or unsubscription', + `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', + `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', + `topic_name` LowCardinality(String) COMMENT 'Name of the topic', + `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the subscription', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, control_index, meta_client_name) +COMMENT 'Contains the details of the RPC subscriptions from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_send_rpc_local ON CLUSTER '{cluster}' +( + `unique_key` Int64 COMMENT 'Unique identifier for each record', + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), + `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer receiver', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, unique_key, meta_client_name) +COMMENT 'Contains the details of the RPC messages sent by the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_synthetic_heartbeat_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'Timestamp of the heartbeat event' CODEC(DoubleDelta, ZSTD(1)), + `remote_peer_id_unique_key` Int64 COMMENT 'Unique key of the remote peer', + `remote_maddrs` String COMMENT 'Multiaddress of the remote peer' CODEC(ZSTD(1)), + `latency_ms` Nullable(Int64) COMMENT 'EWMA latency in milliseconds (0 if unavailable)' CODEC(ZSTD(1)), + `direction` LowCardinality(String) COMMENT 'Connection direction (Unknown/Inbound/Outbound)', + `protocols` Array(String) COMMENT 'List of supported protocols' CODEC(ZSTD(1)), + `connection_age_ms` Nullable(Int64) COMMENT 'Connection age in milliseconds' CODEC(ZSTD(1)), + `remote_agent_implementation` LowCardinality(String) COMMENT 'Implementation of the remote peer', + `remote_agent_version` LowCardinality(String) COMMENT 'Version of the remote peer', + `remote_agent_version_major` LowCardinality(String) COMMENT 'Major version of the remote peer', + `remote_agent_version_minor` LowCardinality(String) COMMENT 'Minor version of the remote peer', + `remote_agent_version_patch` LowCardinality(String) COMMENT 'Patch version of the remote peer', + `remote_agent_platform` LowCardinality(String) COMMENT 'Platform of the remote peer', + `remote_ip` Nullable(IPv6) COMMENT 'IP address of the remote peer' CODEC(ZSTD(1)), + `remote_port` Nullable(UInt16) COMMENT 'Port of the remote peer' CODEC(ZSTD(1)), + `remote_geo_city` LowCardinality(String) COMMENT 'City of the remote peer' CODEC(ZSTD(1)), + `remote_geo_country` LowCardinality(String) COMMENT 'Country of the remote peer' CODEC(ZSTD(1)), + `remote_geo_country_code` LowCardinality(String) COMMENT 'Country code of the remote peer' CODEC(ZSTD(1)), + `remote_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the remote peer' CODEC(ZSTD(1)), + `remote_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the remote peer' CODEC(ZSTD(1)), + `remote_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the remote peer' CODEC(ZSTD(1)), + `remote_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'ASN of the remote peer' CODEC(ZSTD(1)), + `remote_geo_autonomous_system_organization` Nullable(String) COMMENT 'AS organization of the remote peer' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'ASN of the client' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'AS organization of the client' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, remote_peer_id_unique_key, updated_date_time) +COMMENT 'Contains heartbeat events from libp2p peers'; + +CREATE TABLE IF NOT EXISTS default.mempool_dumpster_transaction_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'When this row was last updated, this is outside the source data and used for deduplication' CODEC(DoubleDelta, ZSTD(1)), + `timestamp` DateTime64(3) COMMENT 'Timestamp of the transaction' CODEC(DoubleDelta, ZSTD(1)), + `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), + `chain_id` UInt32 COMMENT 'The chain id of the transaction' CODEC(ZSTD(1)), + `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), + `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), + `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), + `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), + `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), + `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), + `gas_tip_cap` Nullable(UInt128) COMMENT 'The gas tip cap of the transaction in wei' CODEC(ZSTD(1)), + `gas_fee_cap` Nullable(UInt128) COMMENT 'The gas fee cap of the transaction in wei' CODEC(ZSTD(1)), + `data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), + `data_4bytes` Nullable(FixedString(10)) COMMENT 'The first 4 bytes of the call data of the transaction' CODEC(ZSTD(1)), + `sources` Array(LowCardinality(String)) COMMENT 'The sources that saw this transaction in their mempool', + `included_at_block_height` Nullable(UInt64) COMMENT 'The block height at which this transaction was included' CODEC(ZSTD(1)), + `included_block_timestamp` Nullable(DateTime64(3)) COMMENT 'The timestamp of the block at which this transaction was included' CODEC(DoubleDelta, ZSTD(1)), + `inclusion_delay_ms` Nullable(Int64) COMMENT 'The delay between the transaction timestamp and the block timestamp' CODEC(ZSTD(1)) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (chain_id, toYYYYMM(timestamp)) +ORDER BY (chain_id, timestamp, hash, from, nonce, gas) +COMMENT 'Contains transactions from mempool dumpster dataset. Following the parquet schema with some additions'; + +CREATE TABLE IF NOT EXISTS default.mempool_transaction_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'The time when the sentry saw the transaction in the mempool' CODEC(DoubleDelta, ZSTD(1)), + `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), + `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), + `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), + `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), + `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), + `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), + `gas_tip_cap` Nullable(UInt128) COMMENT 'The priority fee (tip) the user has set for the transaction', + `gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction', + `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), + `type` Nullable(UInt8) COMMENT 'The type of the transaction', + `size` UInt32 COMMENT 'The size of the transaction data in bytes' CODEC(ZSTD(1)), + `call_data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), + `blob_gas` Nullable(UInt64) COMMENT 'The maximum gas provided for the blob transaction execution', + `blob_gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction', + `blob_hashes` Array(String) COMMENT 'The hashes of the blob commitments for blob transactions', + `blob_sidecars_size` Nullable(UInt32) COMMENT 'The total size of the sidecars for blob transactions in bytes', + `blob_sidecars_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the sidecars for blob transactions in bytes', + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `meta_execution_fork_id_hash` LowCardinality(String) COMMENT 'The hash of the fork ID of the current Ethereum network', + `meta_execution_fork_id_next` LowCardinality(String) COMMENT 'The fork ID of the next planned Ethereum network upgrade' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, hash, from, nonce, gas) +COMMENT 'Each row represents a transaction that was seen in the mempool by a sentry client. Sentries can report the same transaction multiple times if it has been long enough since the last report.'; + +CREATE TABLE IF NOT EXISTS default.mev_relay_bid_trace_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the bid was fetched' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number within the block bid' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The start time for the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number derived from the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The start time for the epoch that the bid is for' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_request_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_request_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_request_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_request_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `requested_at_slot_time` UInt32 COMMENT 'The time in the slot when the request was sent' CODEC(ZSTD(1)), + `response_at_slot_time` UInt32 COMMENT 'The time in the slot when the response was received' CODEC(ZSTD(1)), + `relay_name` String COMMENT 'The relay that the bid was fetched from' CODEC(ZSTD(1)), + `parent_hash` FixedString(66) COMMENT 'The parent hash of the bid' CODEC(ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number of the bid' CODEC(DoubleDelta, ZSTD(1)), + `block_hash` FixedString(66) COMMENT 'The block hash of the bid' CODEC(ZSTD(1)), + `builder_pubkey` String COMMENT 'The builder pubkey of the bid' CODEC(ZSTD(1)), + `proposer_pubkey` String COMMENT 'The proposer pubkey of the bid' CODEC(ZSTD(1)), + `proposer_fee_recipient` FixedString(42) COMMENT 'The proposer fee recipient of the bid' CODEC(ZSTD(1)), + `gas_limit` UInt64 COMMENT 'The gas limit of the bid' CODEC(DoubleDelta, ZSTD(1)), + `gas_used` UInt64 COMMENT 'The gas used of the bid' CODEC(DoubleDelta, ZSTD(1)), + `value` UInt256 COMMENT 'The transaction value in float64' CODEC(ZSTD(1)), + `num_tx` UInt32 COMMENT 'The number of transactions in the bid' CODEC(DoubleDelta, ZSTD(1)), + `timestamp` Int64 COMMENT 'The timestamp of the bid' CODEC(DoubleDelta, ZSTD(1)), + `timestamp_ms` Int64 COMMENT 'The timestamp of the bid in milliseconds' CODEC(DoubleDelta, ZSTD(1)), + `optimistic_submission` Bool COMMENT 'Whether the bid was optimistic' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, relay_name, block_hash, meta_client_name, builder_pubkey, proposer_pubkey) +COMMENT 'Contains MEV relay block bids data.'; + +CREATE TABLE IF NOT EXISTS default.mev_relay_proposer_payload_delivered_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the payload was delivered' CODEC(DoubleDelta, ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number within the payload' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The start time for the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number derived from the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The start time for the epoch that the bid is for' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number of the payload' CODEC(DoubleDelta, ZSTD(1)), + `relay_name` String COMMENT 'The relay that delivered the payload' CODEC(ZSTD(1)), + `block_hash` FixedString(66) COMMENT 'The block hash associated with the payload' CODEC(ZSTD(1)), + `proposer_pubkey` String COMMENT 'The proposer pubkey that received the payload' CODEC(ZSTD(1)), + `builder_pubkey` String COMMENT 'The builder pubkey that sent the payload' CODEC(ZSTD(1)), + `proposer_fee_recipient` FixedString(42) COMMENT 'The proposer fee recipient of the payload' CODEC(ZSTD(1)), + `gas_limit` UInt64 COMMENT 'The gas limit of the payload' CODEC(DoubleDelta, ZSTD(1)), + `gas_used` UInt64 COMMENT 'The gas used by the payload' CODEC(DoubleDelta, ZSTD(1)), + `value` UInt256 COMMENT 'The bid value in wei' CODEC(ZSTD(1)), + `num_tx` UInt32 COMMENT 'The number of transactions in the payload' CODEC(DoubleDelta, ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) +ORDER BY (meta_network_name, slot_start_date_time, relay_name, block_hash, meta_client_name, builder_pubkey, proposer_pubkey) +COMMENT 'Contains MEV relay proposer payload delivered data.'; + +CREATE TABLE IF NOT EXISTS default.mev_relay_validator_registration_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the registration was fetched' CODEC(DoubleDelta, ZSTD(1)), + `timestamp` Int64 COMMENT 'The timestamp of the registration' CODEC(DoubleDelta, ZSTD(1)), + `relay_name` String COMMENT 'The relay that the registration was fetched from' CODEC(ZSTD(1)), + `validator_index` UInt32 COMMENT 'The validator index of the validator registration' CODEC(ZSTD(1)), + `gas_limit` UInt64 COMMENT 'The gas limit of the validator registration' CODEC(DoubleDelta, ZSTD(1)), + `fee_recipient` String COMMENT 'The fee recipient of the validator registration' CODEC(ZSTD(1)), + `slot` UInt32 COMMENT 'Slot number derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), + `slot_start_date_time` DateTime COMMENT 'The slot start time derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), + `epoch` UInt32 COMMENT 'Epoch number derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), + `epoch_start_date_time` DateTime COMMENT 'The epoch start time derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, meta_client_name, relay_name, validator_index, timestamp) +COMMENT 'Contains MEV relay validator registrations data.'; + +CREATE TABLE IF NOT EXISTS default.node_record_consensus_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the discovery module found the node' CODEC(DoubleDelta, ZSTD(1)), + `enr` String COMMENT 'Ethereum Node Record as text' CODEC(ZSTD(1)), + `node_id` String COMMENT 'ID of the Ethereum Node Record' CODEC(ZSTD(1)), + `peer_id_unique_key` Nullable(Int64) COMMENT 'Unique key associated with the identifier of the peer', + `timestamp` Int64 COMMENT 'Event timestamp in unix time' CODEC(DoubleDelta, ZSTD(1)), + `name` String COMMENT 'Consensus client name' CODEC(ZSTD(1)), + `version` LowCardinality(String) COMMENT 'Consensus client version' CODEC(ZSTD(1)), + `version_major` LowCardinality(String) COMMENT 'Consensus client major version' CODEC(ZSTD(1)), + `version_minor` LowCardinality(String) COMMENT 'Consensus client minor version' CODEC(ZSTD(1)), + `version_patch` LowCardinality(String) COMMENT 'Consensus client patch version' CODEC(ZSTD(1)), + `implementation` LowCardinality(String) COMMENT 'Consensus client implementation' CODEC(ZSTD(1)), + `fork_digest` String COMMENT 'Fork digest value' CODEC(ZSTD(1)), + `next_fork_digest` Nullable(String) COMMENT 'Next fork digest of the next scheduled fork' CODEC(ZSTD(1)), + `finalized_root` String COMMENT 'Finalized beacon block root' CODEC(ZSTD(1)), + `finalized_epoch` UInt64 COMMENT 'Finalized epoch number' CODEC(DoubleDelta, ZSTD(1)), + `head_root` String COMMENT 'Head beacon block root' CODEC(ZSTD(1)), + `head_slot` UInt64 COMMENT 'Head slot number' CODEC(DoubleDelta, ZSTD(1)), + `cgc` Nullable(String) COMMENT 'Represents the nodes custody group count' CODEC(ZSTD(1)), + `finalized_epoch_start_date_time` Nullable(DateTime) COMMENT 'Finalized epoch start time' CODEC(DoubleDelta, ZSTD(1)), + `head_slot_start_date_time` Nullable(DateTime) COMMENT 'Head slot start time' CODEC(DoubleDelta, ZSTD(1)), + `ip` Nullable(IPv6) COMMENT 'IP address of the consensus node' CODEC(ZSTD(1)), + `tcp` Nullable(UInt16) COMMENT 'TCP port from ENR' CODEC(DoubleDelta, ZSTD(1)), + `udp` Nullable(UInt16) COMMENT 'UDP port from ENR' CODEC(DoubleDelta, ZSTD(1)), + `quic` Nullable(UInt16) COMMENT 'QUIC port from ENR' CODEC(DoubleDelta, ZSTD(1)), + `has_ipv6` Bool COMMENT 'Whether the consensus node has an IPv6 address' CODEC(ZSTD(1)), + `geo_city` LowCardinality(String) COMMENT 'City of the consensus node' CODEC(ZSTD(1)), + `geo_country` LowCardinality(String) COMMENT 'Country of the consensus node' CODEC(ZSTD(1)), + `geo_country_code` LowCardinality(String) COMMENT 'Country code of the consensus node' CODEC(ZSTD(1)), + `geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the consensus node' CODEC(ZSTD(1)), + `geo_longitude` Nullable(Float64) COMMENT 'Longitude of the consensus node' CODEC(ZSTD(1)), + `geo_latitude` Nullable(Float64) COMMENT 'Latitude of the consensus node' CODEC(ZSTD(1)), + `geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the consensus node' CODEC(ZSTD(1)), + `geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the consensus node' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, enr, meta_client_name) +COMMENT 'Contains consensus node records discovered by the Xatu discovery module.'; + +CREATE TABLE IF NOT EXISTS default.node_record_execution_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), + `event_date_time` DateTime64(3) COMMENT 'When the event was generated' CODEC(DoubleDelta, ZSTD(1)), + `enr` String COMMENT 'Ethereum Node Record as text' CODEC(ZSTD(1)), + `name` String COMMENT 'Execution client name' CODEC(ZSTD(1)), + `version` LowCardinality(String) COMMENT 'Execution client version' CODEC(ZSTD(1)), + `version_major` LowCardinality(String) COMMENT 'Execution client major version' CODEC(ZSTD(1)), + `version_minor` LowCardinality(String) COMMENT 'Execution client minor version' CODEC(ZSTD(1)), + `version_patch` LowCardinality(String) COMMENT 'Execution client patch version' CODEC(ZSTD(1)), + `implementation` LowCardinality(String) COMMENT 'Execution client implementation' CODEC(ZSTD(1)), + `capabilities` Array(String) COMMENT 'List of capabilities (e.g., eth/65,eth/66)' CODEC(ZSTD(1)), + `protocol_version` String COMMENT 'Protocol version' CODEC(ZSTD(1)), + `total_difficulty` String COMMENT 'Total difficulty of the chain' CODEC(ZSTD(1)), + `head` String COMMENT 'Head block hash' CODEC(ZSTD(1)), + `genesis` String COMMENT 'Genesis block hash' CODEC(ZSTD(1)), + `fork_id_hash` String COMMENT 'Fork ID hash' CODEC(ZSTD(1)), + `fork_id_next` String COMMENT 'Fork ID next block' CODEC(ZSTD(1)), + `node_id` String COMMENT 'Node ID from ENR' CODEC(ZSTD(1)), + `ip` Nullable(IPv6) COMMENT 'IP address of the execution node' CODEC(ZSTD(1)), + `tcp` Nullable(UInt16) COMMENT 'TCP port from ENR' CODEC(DoubleDelta, ZSTD(1)), + `udp` Nullable(UInt16) COMMENT 'UDP port from ENR' CODEC(DoubleDelta, ZSTD(1)), + `has_ipv6` Bool COMMENT 'Whether the execution node has an IPv6 address' CODEC(ZSTD(1)), + `geo_city` LowCardinality(String) COMMENT 'City of the execution node' CODEC(ZSTD(1)), + `geo_country` LowCardinality(String) COMMENT 'Country of the execution node' CODEC(ZSTD(1)), + `geo_country_code` LowCardinality(String) COMMENT 'Country code of the execution node' CODEC(ZSTD(1)), + `geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the execution node' CODEC(ZSTD(1)), + `geo_longitude` Nullable(Float64) COMMENT 'Longitude of the execution node' CODEC(ZSTD(1)), + `geo_latitude` Nullable(Float64) COMMENT 'Latitude of the execution node' CODEC(ZSTD(1)), + `geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the execution node' CODEC(ZSTD(1)), + `geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the execution node' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', + `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', + `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', + `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', + `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), + `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_date_time)) +ORDER BY (meta_network_name, event_date_time, node_id, meta_client_name) +COMMENT 'Contains execution node records discovered by the Xatu discovery module.'; + +-- observoor database + +CREATE TABLE IF NOT EXISTS observoor.block_merge_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `device_id` UInt32 CODEC(ZSTD(1)), + `rw` LowCardinality(String), + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, device_id, rw) +COMMENT 'Aggregated block device I/O merge metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.cpu_utilization_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `total_on_cpu_ns` Float32 CODEC(ZSTD(1)), + `event_count` UInt32 CODEC(ZSTD(1)), + `active_cores` UInt16 CODEC(ZSTD(1)), + `system_cores` UInt16 CODEC(ZSTD(1)), + `max_core_on_cpu_ns` Float32 CODEC(ZSTD(1)), + `max_core_id` UInt32 CODEC(ZSTD(1)), + `mean_core_pct` Float32 CODEC(ZSTD(1)), + `min_core_pct` Float32 CODEC(ZSTD(1)), + `max_core_pct` Float32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated CPU utilization metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.disk_bytes_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `device_id` UInt32 CODEC(ZSTD(1)), + `rw` LowCardinality(String), + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, device_id, rw) +COMMENT 'Aggregated disk I/O byte metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.disk_latency_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `device_id` UInt32 CODEC(ZSTD(1)), + `rw` LowCardinality(String), + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, device_id, rw) +COMMENT 'Aggregated disk I/O latency metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.disk_queue_depth_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `device_id` UInt32 CODEC(ZSTD(1)), + `rw` LowCardinality(String), + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, device_id, rw) +COMMENT 'Aggregated disk queue depth metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.fd_close_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated file descriptor close metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.fd_open_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated file descriptor open metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.host_specs_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `event_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `host_id` String, + `kernel_release` LowCardinality(String), + `os_name` LowCardinality(String), + `architecture` LowCardinality(String), + `cpu_model` String, + `cpu_vendor` LowCardinality(String), + `cpu_online_cores` UInt16 CODEC(ZSTD(1)), + `cpu_logical_cores` UInt16 CODEC(ZSTD(1)), + `cpu_physical_cores` UInt16 CODEC(ZSTD(1)), + `cpu_performance_cores` UInt16 CODEC(ZSTD(1)), + `cpu_efficiency_cores` UInt16 CODEC(ZSTD(1)), + `cpu_unknown_type_cores` UInt16 CODEC(ZSTD(1)), + `cpu_logical_ids` Array(UInt16), + `cpu_core_ids` Array(Int32), + `cpu_package_ids` Array(Int32), + `cpu_die_ids` Array(Int32), + `cpu_cluster_ids` Array(Int32), + `cpu_core_types` Array(UInt8), + `cpu_core_type_labels` Array(String), + `cpu_online_flags` Array(UInt8), + `cpu_max_freq_khz` Array(UInt64), + `cpu_base_freq_khz` Array(UInt64), + `memory_total_bytes` UInt64 CODEC(ZSTD(1)), + `memory_type` LowCardinality(String), + `memory_speed_mts` UInt32 CODEC(ZSTD(1)), + `memory_dimm_count` UInt16 CODEC(ZSTD(1)), + `memory_dimm_sizes_bytes` Array(UInt64), + `memory_dimm_types` Array(String), + `memory_dimm_speeds_mts` Array(UInt32), + `memory_dimm_configured_speeds_mts` Array(UInt32), + `memory_dimm_locators` Array(String), + `memory_dimm_bank_locators` Array(String), + `memory_dimm_manufacturers` Array(String), + `memory_dimm_part_numbers` Array(String), + `memory_dimm_serials` Array(String), + `disk_count` UInt16 CODEC(ZSTD(1)), + `disk_total_bytes` UInt64 CODEC(ZSTD(1)), + `disk_names` Array(String), + `disk_models` Array(String), + `disk_vendors` Array(String), + `disk_serials` Array(String), + `disk_sizes_bytes` Array(UInt64), + `disk_rotational` Array(UInt8), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_time)) +ORDER BY (meta_network_name, event_time, host_id, meta_client_name) +COMMENT 'Periodic host hardware specification snapshots including CPU, memory, and disk details'; + +CREATE TABLE IF NOT EXISTS observoor.mem_compaction_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated memory compaction metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.mem_reclaim_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated memory reclaim metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.memory_usage_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String), + `sampling_rate` Float32 CODEC(ZSTD(1)), + `vm_size_bytes` UInt64 CODEC(ZSTD(1)), + `vm_rss_bytes` UInt64 CODEC(ZSTD(1)), + `rss_anon_bytes` UInt64 CODEC(ZSTD(1)), + `rss_file_bytes` UInt64 CODEC(ZSTD(1)), + `rss_shmem_bytes` UInt64 CODEC(ZSTD(1)), + `vm_swap_bytes` UInt64 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Periodic memory usage snapshots of Ethereum client processes from /proc/[pid]/status'; + +CREATE TABLE IF NOT EXISTS observoor.net_io_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `port_label` LowCardinality(String), + `direction` LowCardinality(String), + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, port_label, direction) +COMMENT 'Aggregated network I/O metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.oom_kill_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated OOM kill events from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.page_fault_major_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated major page fault metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.page_fault_minor_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated minor page fault metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.process_exit_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated process exit events from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.process_fd_usage_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String), + `sampling_rate` Float32 CODEC(ZSTD(1)), + `open_fds` UInt32 CODEC(ZSTD(1)), + `fd_limit_soft` UInt64 CODEC(ZSTD(1)), + `fd_limit_hard` UInt64 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Periodic file descriptor usage snapshots of Ethereum client processes from /proc/[pid]/fd and /proc/[pid]/limits'; + +CREATE TABLE IF NOT EXISTS observoor.process_io_usage_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String), + `sampling_rate` Float32 CODEC(ZSTD(1)), + `rchar_bytes` UInt64 CODEC(ZSTD(1)), + `wchar_bytes` UInt64 CODEC(ZSTD(1)), + `syscr` UInt64 CODEC(ZSTD(1)), + `syscw` UInt64 CODEC(ZSTD(1)), + `read_bytes` UInt64 CODEC(ZSTD(1)), + `write_bytes` UInt64 CODEC(ZSTD(1)), + `cancelled_write_bytes` Int64 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Periodic I/O usage snapshots of Ethereum client processes from /proc/[pid]/io'; + +CREATE TABLE IF NOT EXISTS observoor.process_sched_usage_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String), + `sampling_rate` Float32 CODEC(ZSTD(1)), + `threads` UInt32 CODEC(ZSTD(1)), + `voluntary_ctxt_switches` UInt64 CODEC(ZSTD(1)), + `nonvoluntary_ctxt_switches` UInt64 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Periodic scheduler usage snapshots of Ethereum client processes from /proc/[pid]/status and /proc/[pid]/sched'; + +CREATE TABLE IF NOT EXISTS observoor.raw_events_local ON CLUSTER '{cluster}' +( + `timestamp_ns` UInt64 COMMENT 'Wall clock time of the event in nanoseconds since Unix epoch' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt64 COMMENT 'Ethereum slot number at the time of the event (from wall clock)' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) COMMENT 'Wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `cl_syncing` Bool COMMENT 'Whether the consensus layer was syncing when this event was captured' CODEC(ZSTD(1)), + `el_optimistic` Bool COMMENT 'Whether the execution layer was in optimistic sync mode when this event was captured' CODEC(ZSTD(1)), + `el_offline` Bool COMMENT 'Whether the execution layer was unreachable when this event was captured' CODEC(ZSTD(1)), + `pid` UInt32 COMMENT 'Process ID of the traced Ethereum client' CODEC(ZSTD(1)), + `tid` UInt32 COMMENT 'Thread ID within the traced process' CODEC(ZSTD(1)), + `event_type` LowCardinality(String) COMMENT 'Type of eBPF event (syscall_read, disk_io, net_tx, etc.)', + `client_type` LowCardinality(String) COMMENT 'Ethereum client implementation (geth, reth, prysm, lighthouse, etc.)', + `latency_ns` UInt64 COMMENT 'Latency in nanoseconds for syscall and disk I/O events' CODEC(ZSTD(1)), + `bytes` Int64 COMMENT 'Byte count for I/O events' CODEC(ZSTD(1)), + `src_port` UInt16 COMMENT 'Source port for network events' CODEC(ZSTD(1)), + `dst_port` UInt16 COMMENT 'Destination port for network events' CODEC(ZSTD(1)), + `fd` Int32 COMMENT 'File descriptor number' CODEC(ZSTD(1)), + `filename` String COMMENT 'Filename for fd_open events' CODEC(ZSTD(1)), + `voluntary` Bool COMMENT 'Whether a context switch was voluntary' CODEC(ZSTD(1)), + `on_cpu_ns` UInt64 COMMENT 'Time spent on CPU in nanoseconds before a context switch' CODEC(ZSTD(1)), + `runqueue_ns` UInt64 COMMENT 'Time spent waiting in the run queue' CODEC(ZSTD(1)), + `off_cpu_ns` UInt64 COMMENT 'Time spent off CPU' CODEC(ZSTD(1)), + `major` Bool COMMENT 'Whether a page fault was a major fault' CODEC(ZSTD(1)), + `address` UInt64 COMMENT 'Faulting address for page fault events' CODEC(ZSTD(1)), + `pages` UInt64 COMMENT 'Number of pages for swap events' CODEC(ZSTD(1)), + `rw` UInt8 COMMENT 'Read (0) or write (1) for disk I/O' CODEC(ZSTD(1)), + `queue_depth` UInt32 COMMENT 'Block device queue depth at time of I/O' CODEC(ZSTD(1)), + `device_id` UInt32 COMMENT 'Block device ID (major:minor encoded)' CODEC(ZSTD(1)), + `tcp_state` UInt8 COMMENT 'New TCP state after state change' CODEC(ZSTD(1)), + `tcp_old_state` UInt8 COMMENT 'Previous TCP state before state change' CODEC(ZSTD(1)), + `tcp_srtt_us` UInt32 COMMENT 'Smoothed RTT in microseconds' CODEC(ZSTD(1)), + `tcp_cwnd` UInt32 COMMENT 'Congestion window size' CODEC(ZSTD(1)), + `exit_code` UInt32 COMMENT 'Process exit code' CODEC(ZSTD(1)), + `target_pid` UInt32 COMMENT 'Target PID for OOM kill events' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the node running the observoor agent', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name (mainnet, holesky, etc.)' +) +ENGINE = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') +PARTITION BY (meta_network_name, toYYYYMM(wallclock_slot_start_date_time)) +ORDER BY (meta_network_name, wallclock_slot_start_date_time, client_type, event_type, pid) +COMMENT 'Raw eBPF events captured from Ethereum client processes, one row per kernel event.'; + +CREATE TABLE IF NOT EXISTS observoor.sched_off_cpu_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated scheduler off-CPU metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.sched_on_cpu_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated scheduler on-CPU metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.sched_runqueue_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated scheduler run queue metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.swap_in_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated swap-in metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.swap_out_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated swap-out metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.sync_state_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) COMMENT 'Version column for ReplacingMergeTree deduplication' CODEC(DoubleDelta, ZSTD(1)), + `event_time` DateTime64(3) COMMENT 'Time when the sync state was sampled' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot` UInt32 COMMENT 'Ethereum slot number at sampling time' CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) COMMENT 'Wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), + `cl_syncing` Bool COMMENT 'Whether the consensus layer is syncing' CODEC(ZSTD(1)), + `el_optimistic` Bool COMMENT 'Whether the execution layer is in optimistic sync mode' CODEC(ZSTD(1)), + `el_offline` Bool COMMENT 'Whether the execution layer is unreachable' CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String) COMMENT 'Name of the node running the observoor agent', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name (mainnet, holesky, etc.)' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(event_time)) +ORDER BY (meta_network_name, event_time, meta_client_name) +COMMENT 'Sync state snapshots for consensus and execution layers'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_epoll_wait_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated epoll_wait syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_fdatasync_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated fdatasync syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_fsync_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated fsync syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_futex_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated futex syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_mmap_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated mmap syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_pwrite_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated pwrite syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_read_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated read syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_write_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `histogram` Tuple( le_1us UInt32, le_10us UInt32, le_100us UInt32, le_1ms UInt32, le_10ms UInt32, le_100ms UInt32, le_1s UInt32, le_10s UInt32, le_100s UInt32, inf UInt32) CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated write syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.tcp_cwnd_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `port_label` LowCardinality(String), + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, port_label) +COMMENT 'Aggregated TCP congestion window metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.tcp_retransmit_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `port_label` LowCardinality(String), + `direction` LowCardinality(String), + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, port_label, direction) +COMMENT 'Aggregated TCP retransmit metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.tcp_rtt_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `port_label` LowCardinality(String), + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `min` Float32 CODEC(ZSTD(1)), + `max` Float32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, port_label) +COMMENT 'Aggregated TCP round-trip time metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.tcp_state_change_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `window_start` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `interval_ms` UInt16 CODEC(ZSTD(1)), + `wallclock_slot` UInt32 CODEC(DoubleDelta, ZSTD(1)), + `wallclock_slot_start_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `pid` UInt32 CODEC(ZSTD(1)), + `client_type` LowCardinality(String), + `sampling_mode` LowCardinality(String) DEFAULT 'none', + `sampling_rate` Float32 DEFAULT 1., + `sum` Float32 CODEC(ZSTD(1)), + `count` UInt32 CODEC(ZSTD(1)), + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String) +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY (meta_network_name, toYYYYMM(window_start)) +ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type) +COMMENT 'Aggregated TCP state change events from eBPF tracing of Ethereum client processes'; + +-- admin database + +CREATE TABLE IF NOT EXISTS admin.cryo_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), + `dataset` LowCardinality(String), + `mode` LowCardinality(String), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY meta_network_name +ORDER BY (dataset, mode, meta_network_name) +COMMENT 'Tracks cryo dataset processing state per block'; + +CREATE TABLE IF NOT EXISTS admin.execution_block_local ON CLUSTER '{cluster}' +( + `updated_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), + `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), + `processor` LowCardinality(String) COMMENT 'The type of processor that processed the block', + `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', + `complete` UInt8, + `task_count` UInt32 +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) +PARTITION BY meta_network_name +ORDER BY (block_number, processor, meta_network_name) +COMMENT 'Tracks execution block processing state'; + +-- DISTRIBUTED TABLES +-- default database + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_beacon_blob ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_beacon_blob_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_beacon_blob_local', cityHash64(slot_start_date_time, meta_client_name, block_root)) +COMMENT 'Contains beacon API blob metadata derived from block blob_kzg_commitments from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_beacon_committee_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_beacon_committee_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, committee_index)) +COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_attestation ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_attestation_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_attestation_local', cityHash64(slot_start_date_time, meta_client_name)) +COMMENT 'Contains beacon API eventstream "attestation" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_blob_sidecar_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_blob_sidecar_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, block_root, blob_index)) +COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_block_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_block_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, block)) +COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_block_gossip ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_block_gossip_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_block_gossip_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, block)) +COMMENT 'Contains beacon API eventstream "block_gossip" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_chain_reorg_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_chain_reorg_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, old_head_block, new_head_block)) +COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_contribution_and_proof_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_contribution_and_proof_local', cityHash64(contribution_slot_start_date_time, meta_network_name, meta_client_name, contribution_beacon_block_root, contribution_subcommittee_index, signature)) +COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_data_column_sidecar ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_data_column_sidecar_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_data_column_sidecar_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, block_root, column_index)) +COMMENT 'Contains beacon API eventstream "data_column_sidecar" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_finalized_checkpoint_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_finalized_checkpoint_local', cityHash64(epoch_start_date_time, meta_network_name, meta_client_name, block, state)) +COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_head_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_head_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, block, previous_duty_dependent_root, current_duty_dependent_root)) +COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures head events. Each row represents a `head` event from the Beacon API `/eth/v1/events?topics=head`, indicating the chain''s canonical head has been updated. Sentry adds client metadata and propagation timing. Partition: monthly by `slot_start_date_time`.'''; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_events_voluntary_exit_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_events_voluntary_exit_local', cityHash64(wallclock_epoch_start_date_time, meta_network_name, meta_client_name, validator_index)) +COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_proposer_duty_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_proposer_duty_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, proposer_validator_index)) +COMMENT 'Contains a proposer duty from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v1_validator_attestation_data_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v1_validator_attestation_data_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, committee_index, beacon_block_root, source_root, target_root)) +COMMENT 'Contains beacon API validator attestation data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v2_beacon_block_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v2_beacon_block_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, block_root, parent_root, state_root)) +COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v3_validator_block ON CLUSTER '{cluster}' +AS default.beacon_api_eth_v3_validator_block_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_eth_v3_validator_block_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, event_date_time)) +COMMENT 'Contains beacon API /eth/v3/validator/blocks/{slot} data from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_api_slot ON CLUSTER '{cluster}' +AS default.beacon_api_slot_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_api_slot_local', cityHash64(slot_start_date_time, slot)) +COMMENT 'Aggregated beacon API slot data. Each row represents a slot from each sentry client attached to a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.beacon_block_classification ON CLUSTER '{cluster}' +AS default.beacon_block_classification_local +ENGINE = Distributed('{cluster}', 'default', 'beacon_block_classification_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, proposer_index)) +COMMENT 'Contains beacon block classification for a given slot. This is a best guess based on the client probabilities of the proposer. This is not guaranteed to be correct.'; + +CREATE TABLE IF NOT EXISTS default.blob_submitter ON CLUSTER '{cluster}' +AS default.blob_submitter_local +ENGINE = Distributed('{cluster}', 'default', 'blob_submitter_local', cityHash64(address, meta_network_name)) +COMMENT 'Contains blob submitter address to name mappings.'; + +CREATE TABLE IF NOT EXISTS default.block_native_mempool_transaction ON CLUSTER '{cluster}' +AS default.block_native_mempool_transaction_local +ENGINE = Distributed('{cluster}', 'default', 'block_native_mempool_transaction_local', cityHash64(detecttime, network, hash, fromaddress, nonce, gas)) +COMMENT 'Contains transactions from block native mempool dataset'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' +AS default.canonical_beacon_blob_sidecar_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_blob_sidecar_local', cityHash64(slot_start_date_time, meta_network_name, block_root, blob_index)) +COMMENT 'Contains a blob sidecar from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_local', cityHash64(slot_start_date_time, meta_network_name)) +COMMENT 'Contains beacon block from a beacon node.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_attester_slashing_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_attester_slashing_local', cityHash64(slot_start_date_time, meta_network_name, block_root, attestation_1_attesting_indices, attestation_2_attesting_indices, attestation_1_data_slot, attestation_2_data_slot, attestation_1_data_beacon_block_root, attestation_2_data_beacon_block_root)) +COMMENT 'Contains attester slashing from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_bls_to_execution_change_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_bls_to_execution_change_local', cityHash64(slot_start_date_time, meta_network_name, block_root, exchanging_message_validator_index, exchanging_message_from_bls_pubkey, exchanging_message_to_execution_address)) +COMMENT 'Contains bls to execution change from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_deposit_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_deposit_local', cityHash64(slot_start_date_time, meta_network_name, block_root, deposit_data_pubkey, deposit_proof)) +COMMENT 'Contains a deposit from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_execution_transaction_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_execution_transaction_local', cityHash64(slot_start_date_time, meta_network_name, block_root, position, hash, nonce)) +COMMENT 'Contains execution transaction from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_proposer_slashing_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_proposer_slashing_local', cityHash64(slot_start_date_time, meta_network_name, block_root, signed_header_1_message_slot, signed_header_2_message_slot, signed_header_1_message_proposer_index, signed_header_2_message_proposer_index, signed_header_1_message_body_root, signed_header_2_message_body_root)) +COMMENT 'Contains proposer slashing from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_sync_aggregate ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_sync_aggregate_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_sync_aggregate_local', cityHash64(slot_start_date_time, meta_network_name, slot)) +COMMENT 'Contains canonical beacon block sync aggregate data with expanded validator participation.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_voluntary_exit_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_voluntary_exit_local', cityHash64(slot_start_date_time, meta_network_name, block_root, voluntary_exit_message_epoch, voluntary_exit_message_validator_index)) +COMMENT 'Contains a voluntary exit from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' +AS default.canonical_beacon_block_withdrawal_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_block_withdrawal_local', cityHash64(slot_start_date_time, meta_network_name, block_root, withdrawal_index, withdrawal_validator_index)) +COMMENT 'Contains a withdrawal from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_committee ON CLUSTER '{cluster}' +AS default.canonical_beacon_committee_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_committee_local', cityHash64(slot_start_date_time, meta_network_name, committee_index)) +COMMENT 'Contains canonical beacon API /eth/v1/beacon/committees data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' +AS default.canonical_beacon_elaborated_attestation_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_elaborated_attestation_local', cityHash64(slot_start_date_time, meta_network_name, block_root, block_slot, position_in_block, beacon_block_root, slot, committee_index, source_root, target_root)) +COMMENT 'Contains elaborated attestations from beacon blocks.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' +AS default.canonical_beacon_proposer_duty_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_proposer_duty_local', cityHash64(slot_start_date_time, meta_network_name, proposer_validator_index, proposer_pubkey)) +COMMENT 'Contains a proposer duty from a beacon block.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_sync_committee ON CLUSTER '{cluster}' +AS default.canonical_beacon_sync_committee_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_sync_committee_local', cityHash64(epoch_start_date_time, meta_network_name, sync_committee_period)) +COMMENT 'Contains canonical beacon API /eth/v1/beacon/states/{state_id}/sync_committees data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_validators ON CLUSTER '{cluster}' +AS default.canonical_beacon_validators_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_validators_local', cityHash64(epoch_start_date_time, meta_network_name, index, status)) +COMMENT 'Contains a validator state for an epoch.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_validators_pubkeys ON CLUSTER '{cluster}' +AS default.canonical_beacon_validators_pubkeys_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_validators_pubkeys_local', cityHash64(index, meta_network_name)) +COMMENT 'Contains a validator pubkeys for an epoch.'; + +CREATE TABLE IF NOT EXISTS default.canonical_beacon_validators_withdrawal_credentials ON CLUSTER '{cluster}' +AS default.canonical_beacon_validators_withdrawal_credentials_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_beacon_validators_withdrawal_credentials_local', cityHash64(index, meta_network_name)) +COMMENT 'Contains a validator withdrawal credentials for an epoch.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_address_appearances ON CLUSTER '{cluster}' +AS default.canonical_execution_address_appearances_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_address_appearances_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution address appearance data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_balance_diffs ON CLUSTER '{cluster}' +AS default.canonical_execution_balance_diffs_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_balance_diffs_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution balance diff data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_balance_reads ON CLUSTER '{cluster}' +AS default.canonical_execution_balance_reads_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_balance_reads_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution balance read data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_block ON CLUSTER '{cluster}' +AS default.canonical_execution_block_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_block_local', cityHash64(block_number, meta_network_name)) +COMMENT 'Contains canonical execution block data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_contracts ON CLUSTER '{cluster}' +AS default.canonical_execution_contracts_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_contracts_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution contract data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_erc20_transfers ON CLUSTER '{cluster}' +AS default.canonical_execution_erc20_transfers_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_erc20_transfers_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution erc20 transfer data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_erc721_transfers ON CLUSTER '{cluster}' +AS default.canonical_execution_erc721_transfers_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_erc721_transfers_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution erc721 transfer data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_four_byte_counts ON CLUSTER '{cluster}' +AS default.canonical_execution_four_byte_counts_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_four_byte_counts_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution four byte count data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_logs ON CLUSTER '{cluster}' +AS default.canonical_execution_logs_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_logs_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution logs data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_native_transfers ON CLUSTER '{cluster}' +AS default.canonical_execution_native_transfers_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_native_transfers_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution native transfer data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_nonce_diffs ON CLUSTER '{cluster}' +AS default.canonical_execution_nonce_diffs_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_nonce_diffs_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution nonce diff data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_nonce_reads ON CLUSTER '{cluster}' +AS default.canonical_execution_nonce_reads_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_nonce_reads_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution nonce read data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_storage_diffs ON CLUSTER '{cluster}' +AS default.canonical_execution_storage_diffs_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_storage_diffs_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution storage diffs data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_storage_reads ON CLUSTER '{cluster}' +AS default.canonical_execution_storage_reads_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_storage_reads_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution storage reads data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_traces ON CLUSTER '{cluster}' +AS default.canonical_execution_traces_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_traces_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution traces data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_transaction ON CLUSTER '{cluster}' +AS default.canonical_execution_transaction_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_transaction_local', cityHash64(block_number, meta_network_name, transaction_hash)) +COMMENT 'Contains canonical execution transaction data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_transaction_structlog ON CLUSTER '{cluster}' +AS default.canonical_execution_transaction_structlog_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_transaction_structlog_local', cityHash64(block_number, meta_network_name, transaction_hash, index)) +COMMENT 'Contains canonical execution transaction structlog data.'; + +CREATE TABLE IF NOT EXISTS default.canonical_execution_transaction_structlog_agg ON CLUSTER '{cluster}' +AS default.canonical_execution_transaction_structlog_agg_local +ENGINE = Distributed('{cluster}', 'default', 'canonical_execution_transaction_structlog_agg_local', cityHash64(block_number, meta_network_name, transaction_hash, call_frame_id)) +COMMENT 'Aggregated EVM execution data. Summary rows (operation="") contain frame metadata. Per-opcode rows contain aggregated gas/count per (frame, opcode).'; + +CREATE TABLE IF NOT EXISTS default.consensus_engine_api_get_blobs ON CLUSTER '{cluster}' +AS default.consensus_engine_api_get_blobs_local +ENGINE = Distributed('{cluster}', 'default', 'consensus_engine_api_get_blobs_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, block_root, event_date_time)) +COMMENT 'Contains timing and instrumentation data for engine_getBlobs calls between the consensus and execution layer.'; + +CREATE TABLE IF NOT EXISTS default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' +AS default.consensus_engine_api_new_payload_local +ENGINE = Distributed('{cluster}', 'default', 'consensus_engine_api_new_payload_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, block_hash, event_date_time)) +COMMENT 'Contains timing and instrumentation data for engine_newPayload calls between the consensus and execution layer.'; + +CREATE TABLE IF NOT EXISTS default.ethseer_validator_entity ON CLUSTER '{cluster}' +AS default.ethseer_validator_entity_local +ENGINE = Distributed('{cluster}', 'default', 'ethseer_validator_entity_local', cityHash64(index, pubkey, meta_network_name)) +COMMENT 'Contains a mapping of validators to entities'; + +CREATE TABLE IF NOT EXISTS default.execution_block_metrics ON CLUSTER '{cluster}' +AS default.execution_block_metrics_local +ENGINE = Distributed('{cluster}', 'default', 'execution_block_metrics_local', cityHash64(block_number, meta_network_name, meta_client_name)) +COMMENT 'Contains detailed performance metrics from execution client structured logging for block execution'; + +CREATE TABLE IF NOT EXISTS default.execution_engine_get_blobs ON CLUSTER '{cluster}' +AS default.execution_engine_get_blobs_local +ENGINE = Distributed('{cluster}', 'default', 'execution_engine_get_blobs_local', cityHash64(event_date_time, meta_network_name, meta_client_name)) +COMMENT 'Contains timing and instrumentation data for engine_getBlobs calls from the execution layer perspective.'; + +CREATE TABLE IF NOT EXISTS default.execution_engine_new_payload ON CLUSTER '{cluster}' +AS default.execution_engine_new_payload_local +ENGINE = Distributed('{cluster}', 'default', 'execution_engine_new_payload_local', cityHash64(block_number, meta_network_name, meta_client_name, block_hash, event_date_time)) +COMMENT 'Contains timing and instrumentation data for engine_newPayload calls from the execution layer perspective.'; + +CREATE TABLE IF NOT EXISTS default.execution_state_size ON CLUSTER '{cluster}' +AS default.execution_state_size_local +ENGINE = Distributed('{cluster}', 'default', 'execution_state_size_local', cityHash64(block_number, meta_network_name, meta_client_name, state_root, event_date_time)) +COMMENT 'Contains execution layer state size metrics including account, contract code, and storage data measurements at specific block heights.'; + +CREATE TABLE IF NOT EXISTS default.execution_transaction ON CLUSTER '{cluster}' +AS default.execution_transaction_local +ENGINE = Distributed('{cluster}', 'default', 'execution_transaction_local', cityHash64(block_number, meta_network_name, block_hash, position)) +COMMENT 'Contains execution transaction data that may not be canonical.'; + +CREATE TABLE IF NOT EXISTS default.imported_sources ON CLUSTER '{cluster}' +AS default.imported_sources_local +ENGINE = Distributed('{cluster}', 'default', 'imported_sources_local', rand()) +COMMENT 'This table contains the list of sources that have been imported into the database'; + +CREATE TABLE IF NOT EXISTS default.libp2p_add_peer ON CLUSTER '{cluster}' +AS default.libp2p_add_peer_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_add_peer_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key)) +COMMENT 'Contains the details of the peers added to the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_connected ON CLUSTER '{cluster}' +AS default.libp2p_connected_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_connected_local', cityHash64(event_date_time, meta_network_name, meta_client_name, remote_peer_id_unique_key, direction, opened)) +COMMENT 'Contains the details of the CONNECTED events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_deliver_message ON CLUSTER '{cluster}' +AS default.libp2p_deliver_message_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_deliver_message_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name, message_id, seq_number)) +COMMENT 'Contains the details of the DELIVER_MESSAGE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_disconnected ON CLUSTER '{cluster}' +AS default.libp2p_disconnected_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_disconnected_local', cityHash64(event_date_time, meta_network_name, meta_client_name, remote_peer_id_unique_key, direction, opened)) +COMMENT 'Contains the details of the DISCONNECTED events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_drop_rpc ON CLUSTER '{cluster}' +AS default.libp2p_drop_rpc_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_drop_rpc_local', unique_key) +COMMENT 'Contains the details of the RPC messages dropped by the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_duplicate_message ON CLUSTER '{cluster}' +AS default.libp2p_duplicate_message_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_duplicate_message_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name, message_id, seq_number)) +COMMENT 'Contains the details of the DUPLICATE_MESSAGE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_aggregate_and_proof ON CLUSTER '{cluster}' +AS default.libp2p_gossipsub_aggregate_and_proof_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_gossipsub_aggregate_and_proof_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, peer_id_unique_key, message_id)) +COMMENT 'Table for libp2p gossipsub aggregate and proof data.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' +AS default.libp2p_gossipsub_beacon_attestation_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_gossipsub_beacon_attestation_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, peer_id_unique_key, message_id)) +COMMENT 'Table for libp2p gossipsub beacon attestation data.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' +AS default.libp2p_gossipsub_beacon_block_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_gossipsub_beacon_block_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, peer_id_unique_key, message_id)) +COMMENT 'Table for libp2p gossipsub beacon block data.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' +AS default.libp2p_gossipsub_blob_sidecar_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_gossipsub_blob_sidecar_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, peer_id_unique_key, message_id)) +COMMENT 'Table for libp2p gossipsub blob sidecar data'; + +CREATE TABLE IF NOT EXISTS default.libp2p_gossipsub_data_column_sidecar ON CLUSTER '{cluster}' +AS default.libp2p_gossipsub_data_column_sidecar_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_gossipsub_data_column_sidecar_local', cityHash64(slot_start_date_time, meta_network_name, meta_client_name, peer_id_unique_key, message_id)) +COMMENT 'Table for libp2p gossipsub data column sidecar data'; + +CREATE TABLE IF NOT EXISTS default.libp2p_graft ON CLUSTER '{cluster}' +AS default.libp2p_graft_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_graft_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name)) +COMMENT 'Contains the details of the GRAFT events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_handle_metadata ON CLUSTER '{cluster}' +AS default.libp2p_handle_metadata_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_handle_metadata_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, latency_milliseconds)) +COMMENT 'Contains the metadata handling events for libp2p peers.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_handle_status ON CLUSTER '{cluster}' +AS default.libp2p_handle_status_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_handle_status_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, latency_milliseconds)) +COMMENT 'Contains the status handling events for libp2p peers.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_identify ON CLUSTER '{cluster}' +AS default.libp2p_identify_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_identify_local', cityHash64(event_date_time, meta_network_name, meta_client_name, remote_peer_id_unique_key, direction)) +COMMENT 'Contains libp2p identify protocol exchange results including remote peer agent info, supported protocols, and connection metadata'; + +CREATE TABLE IF NOT EXISTS default.libp2p_join ON CLUSTER '{cluster}' +AS default.libp2p_join_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_join_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name)) +COMMENT 'Contains the details of the JOIN events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_leave ON CLUSTER '{cluster}' +AS default.libp2p_leave_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_leave_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name)) +COMMENT 'Contains the details of the LEAVE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_peer ON CLUSTER '{cluster}' +AS default.libp2p_peer_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_peer_local', unique_key) +COMMENT 'Contains the original peer id of a seahashed peer_id + meta_network_name, commonly seen in other tables as the field peer_id_unique_key'; + +CREATE TABLE IF NOT EXISTS default.libp2p_prune ON CLUSTER '{cluster}' +AS default.libp2p_prune_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_prune_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name)) +COMMENT 'Contains the details of the PRUNE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_publish_message ON CLUSTER '{cluster}' +AS default.libp2p_publish_message_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_publish_message_local', cityHash64(event_date_time, meta_network_name, meta_client_name, topic_fork_digest_value, topic_name, message_id)) +COMMENT 'Contains the details of the PUBLISH_MESSAGE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_recv_rpc ON CLUSTER '{cluster}' +AS default.libp2p_recv_rpc_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_recv_rpc_local', unique_key) +COMMENT 'Contains the details of the RPC messages received by the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_reject_message ON CLUSTER '{cluster}' +AS default.libp2p_reject_message_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_reject_message_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, topic_fork_digest_value, topic_name, message_id, seq_number)) +COMMENT 'Contains the details of the REJECT_MESSAGE events from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_remove_peer ON CLUSTER '{cluster}' +AS default.libp2p_remove_peer_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_remove_peer_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key)) +COMMENT 'Contains the details of the peers removed from the libp2p client.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_data_column_custody_probe ON CLUSTER '{cluster}' +AS default.libp2p_rpc_data_column_custody_probe_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_rpc_data_column_custody_probe_local', cityHash64(event_date_time, meta_network_name, meta_client_name, peer_id_unique_key, slot, column_index)) +COMMENT 'Contains custody probe events for data column availability verification'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_graft ON CLUSTER '{cluster}' +AS default.libp2p_rpc_meta_control_graft_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_rpc_meta_control_graft_local', unique_key) +COMMENT 'Contains the details of the "Graft" control messages from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_idontwant ON CLUSTER '{cluster}' +AS default.libp2p_rpc_meta_control_idontwant_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_rpc_meta_control_idontwant_local', unique_key) +COMMENT 'Contains the details of the IDONTWANT control messages from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_ihave ON CLUSTER '{cluster}' +AS default.libp2p_rpc_meta_control_ihave_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_rpc_meta_control_ihave_local', unique_key) +COMMENT 'Contains the details of the "I have" control messages from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_iwant ON CLUSTER '{cluster}' +AS default.libp2p_rpc_meta_control_iwant_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_rpc_meta_control_iwant_local', unique_key) +COMMENT 'Contains IWANT control messages from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Peers request specific message IDs. Partition: monthly by `event_date_time`'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_control_prune ON CLUSTER '{cluster}' +AS default.libp2p_rpc_meta_control_prune_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_rpc_meta_control_prune_local', unique_key) +COMMENT 'Contains the details of the "Prune" control messages from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_message ON CLUSTER '{cluster}' +AS default.libp2p_rpc_meta_message_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_rpc_meta_message_local', unique_key) +COMMENT 'Contains the details of the RPC meta messages from the peer'; + +CREATE TABLE IF NOT EXISTS default.libp2p_rpc_meta_subscription ON CLUSTER '{cluster}' +AS default.libp2p_rpc_meta_subscription_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_rpc_meta_subscription_local', unique_key) +COMMENT 'Contains the details of the RPC subscriptions from the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_send_rpc ON CLUSTER '{cluster}' +AS default.libp2p_send_rpc_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_send_rpc_local', unique_key) +COMMENT 'Contains the details of the RPC messages sent by the peer.'; + +CREATE TABLE IF NOT EXISTS default.libp2p_synthetic_heartbeat ON CLUSTER '{cluster}' +AS default.libp2p_synthetic_heartbeat_local +ENGINE = Distributed('{cluster}', 'default', 'libp2p_synthetic_heartbeat_local', cityHash64(event_date_time, meta_network_name, meta_client_name, remote_peer_id_unique_key, updated_date_time)) +COMMENT 'Contains heartbeat events from libp2p peers'; + +CREATE TABLE IF NOT EXISTS default.mempool_dumpster_transaction ON CLUSTER '{cluster}' +AS default.mempool_dumpster_transaction_local +ENGINE = Distributed('{cluster}', 'default', 'mempool_dumpster_transaction_local', cityHash64(timestamp, chain_id, hash, from, nonce, gas)) +COMMENT 'Contains transactions from mempool dumpster dataset. Following the parquet schema with some additions'; + +CREATE TABLE IF NOT EXISTS default.mempool_transaction ON CLUSTER '{cluster}' +AS default.mempool_transaction_local +ENGINE = Distributed('{cluster}', 'default', 'mempool_transaction_local', cityHash64(event_date_time, meta_network_name, meta_client_name, hash, from, nonce, gas)) +COMMENT 'Each row represents a transaction that was seen in the mempool by a sentry client. Sentries can report the same transaction multiple times if it has been long enough since the last report.'; + +CREATE TABLE IF NOT EXISTS default.mev_relay_bid_trace ON CLUSTER '{cluster}' +AS default.mev_relay_bid_trace_local +ENGINE = Distributed('{cluster}', 'default', 'mev_relay_bid_trace_local', cityHash64(slot, meta_network_name)) +COMMENT 'Contains MEV relay block bids data.'; + +CREATE TABLE IF NOT EXISTS default.mev_relay_proposer_payload_delivered ON CLUSTER '{cluster}' +AS default.mev_relay_proposer_payload_delivered_local +ENGINE = Distributed('{cluster}', 'default', 'mev_relay_proposer_payload_delivered_local', cityHash64(slot, meta_network_name)) +COMMENT 'Contains MEV relay proposer payload delivered data.'; + +CREATE TABLE IF NOT EXISTS default.mev_relay_validator_registration ON CLUSTER '{cluster}' +AS default.mev_relay_validator_registration_local +ENGINE = Distributed('{cluster}', 'default', 'mev_relay_validator_registration_local', cityHash64(slot, meta_network_name)) +COMMENT 'Contains MEV relay validator registrations data.'; + +CREATE TABLE IF NOT EXISTS default.node_record_consensus ON CLUSTER '{cluster}' +AS default.node_record_consensus_local +ENGINE = Distributed('{cluster}', 'default', 'node_record_consensus_local', cityHash64(event_date_time, meta_network_name, enr, meta_client_name)) +COMMENT 'Contains consensus node records discovered by the Xatu discovery module.'; + +CREATE TABLE IF NOT EXISTS default.node_record_execution ON CLUSTER '{cluster}' +AS default.node_record_execution_local +ENGINE = Distributed('{cluster}', 'default', 'node_record_execution_local', cityHash64(event_date_time, meta_network_name, node_id, meta_client_name)) +COMMENT 'Contains execution node records discovered by the Xatu discovery module.'; + +-- observoor database + +CREATE TABLE IF NOT EXISTS observoor.block_merge ON CLUSTER '{cluster}' +AS observoor.block_merge_local +ENGINE = Distributed('{cluster}', 'observoor', 'block_merge_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated block device I/O merge metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.cpu_utilization ON CLUSTER '{cluster}' +AS observoor.cpu_utilization_local +ENGINE = Distributed('{cluster}', 'observoor', 'cpu_utilization_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated CPU utilization metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.disk_bytes ON CLUSTER '{cluster}' +AS observoor.disk_bytes_local +ENGINE = Distributed('{cluster}', 'observoor', 'disk_bytes_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated disk I/O byte metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.disk_latency ON CLUSTER '{cluster}' +AS observoor.disk_latency_local +ENGINE = Distributed('{cluster}', 'observoor', 'disk_latency_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated disk I/O latency metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.disk_queue_depth ON CLUSTER '{cluster}' +AS observoor.disk_queue_depth_local +ENGINE = Distributed('{cluster}', 'observoor', 'disk_queue_depth_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated disk queue depth metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.fd_close ON CLUSTER '{cluster}' +AS observoor.fd_close_local +ENGINE = Distributed('{cluster}', 'observoor', 'fd_close_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated file descriptor close metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.fd_open ON CLUSTER '{cluster}' +AS observoor.fd_open_local +ENGINE = Distributed('{cluster}', 'observoor', 'fd_open_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated file descriptor open metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.host_specs ON CLUSTER '{cluster}' +AS observoor.host_specs_local +ENGINE = Distributed('{cluster}', 'observoor', 'host_specs_local', cityHash64(event_time, meta_network_name, host_id, meta_client_name)) +COMMENT 'Periodic host hardware specification snapshots including CPU, memory, and disk details'; + +CREATE TABLE IF NOT EXISTS observoor.mem_compaction ON CLUSTER '{cluster}' +AS observoor.mem_compaction_local +ENGINE = Distributed('{cluster}', 'observoor', 'mem_compaction_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated memory compaction metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.mem_reclaim ON CLUSTER '{cluster}' +AS observoor.mem_reclaim_local +ENGINE = Distributed('{cluster}', 'observoor', 'mem_reclaim_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated memory reclaim metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.memory_usage ON CLUSTER '{cluster}' +AS observoor.memory_usage_local +ENGINE = Distributed('{cluster}', 'observoor', 'memory_usage_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Periodic memory usage snapshots of Ethereum client processes from /proc/[pid]/status'; + +CREATE TABLE IF NOT EXISTS observoor.net_io ON CLUSTER '{cluster}' +AS observoor.net_io_local +ENGINE = Distributed('{cluster}', 'observoor', 'net_io_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated network I/O metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.oom_kill ON CLUSTER '{cluster}' +AS observoor.oom_kill_local +ENGINE = Distributed('{cluster}', 'observoor', 'oom_kill_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated OOM kill events from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.page_fault_major ON CLUSTER '{cluster}' +AS observoor.page_fault_major_local +ENGINE = Distributed('{cluster}', 'observoor', 'page_fault_major_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated major page fault metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.page_fault_minor ON CLUSTER '{cluster}' +AS observoor.page_fault_minor_local +ENGINE = Distributed('{cluster}', 'observoor', 'page_fault_minor_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated minor page fault metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.process_exit ON CLUSTER '{cluster}' +AS observoor.process_exit_local +ENGINE = Distributed('{cluster}', 'observoor', 'process_exit_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated process exit events from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.process_fd_usage ON CLUSTER '{cluster}' +AS observoor.process_fd_usage_local +ENGINE = Distributed('{cluster}', 'observoor', 'process_fd_usage_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Periodic file descriptor usage snapshots of Ethereum client processes from /proc/[pid]/fd and /proc/[pid]/limits'; + +CREATE TABLE IF NOT EXISTS observoor.process_io_usage ON CLUSTER '{cluster}' +AS observoor.process_io_usage_local +ENGINE = Distributed('{cluster}', 'observoor', 'process_io_usage_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Periodic I/O usage snapshots of Ethereum client processes from /proc/[pid]/io'; + +CREATE TABLE IF NOT EXISTS observoor.process_sched_usage ON CLUSTER '{cluster}' +AS observoor.process_sched_usage_local +ENGINE = Distributed('{cluster}', 'observoor', 'process_sched_usage_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Periodic scheduler usage snapshots of Ethereum client processes from /proc/[pid]/status and /proc/[pid]/sched'; + +CREATE TABLE IF NOT EXISTS observoor.raw_events ON CLUSTER '{cluster}' +AS observoor.raw_events_local +ENGINE = Distributed('{cluster}', 'observoor', 'raw_events_local', rand()) +COMMENT 'Raw eBPF events captured from Ethereum client processes, one row per kernel event.'; + +CREATE TABLE IF NOT EXISTS observoor.sched_off_cpu ON CLUSTER '{cluster}' +AS observoor.sched_off_cpu_local +ENGINE = Distributed('{cluster}', 'observoor', 'sched_off_cpu_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated scheduler off-CPU metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.sched_on_cpu ON CLUSTER '{cluster}' +AS observoor.sched_on_cpu_local +ENGINE = Distributed('{cluster}', 'observoor', 'sched_on_cpu_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated scheduler on-CPU metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.sched_runqueue ON CLUSTER '{cluster}' +AS observoor.sched_runqueue_local +ENGINE = Distributed('{cluster}', 'observoor', 'sched_runqueue_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated scheduler run queue metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.swap_in ON CLUSTER '{cluster}' +AS observoor.swap_in_local +ENGINE = Distributed('{cluster}', 'observoor', 'swap_in_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated swap-in metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.swap_out ON CLUSTER '{cluster}' +AS observoor.swap_out_local +ENGINE = Distributed('{cluster}', 'observoor', 'swap_out_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated swap-out metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.sync_state ON CLUSTER '{cluster}' +AS observoor.sync_state_local +ENGINE = Distributed('{cluster}', 'observoor', 'sync_state_local', cityHash64(event_time, meta_network_name, meta_client_name)) +COMMENT 'Sync state snapshots for consensus and execution layers'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_epoll_wait ON CLUSTER '{cluster}' +AS observoor.syscall_epoll_wait_local +ENGINE = Distributed('{cluster}', 'observoor', 'syscall_epoll_wait_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated epoll_wait syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_fdatasync ON CLUSTER '{cluster}' +AS observoor.syscall_fdatasync_local +ENGINE = Distributed('{cluster}', 'observoor', 'syscall_fdatasync_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated fdatasync syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_fsync ON CLUSTER '{cluster}' +AS observoor.syscall_fsync_local +ENGINE = Distributed('{cluster}', 'observoor', 'syscall_fsync_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated fsync syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_futex ON CLUSTER '{cluster}' +AS observoor.syscall_futex_local +ENGINE = Distributed('{cluster}', 'observoor', 'syscall_futex_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated futex syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_mmap ON CLUSTER '{cluster}' +AS observoor.syscall_mmap_local +ENGINE = Distributed('{cluster}', 'observoor', 'syscall_mmap_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated mmap syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_pwrite ON CLUSTER '{cluster}' +AS observoor.syscall_pwrite_local +ENGINE = Distributed('{cluster}', 'observoor', 'syscall_pwrite_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated pwrite syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_read ON CLUSTER '{cluster}' +AS observoor.syscall_read_local +ENGINE = Distributed('{cluster}', 'observoor', 'syscall_read_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated read syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.syscall_write ON CLUSTER '{cluster}' +AS observoor.syscall_write_local +ENGINE = Distributed('{cluster}', 'observoor', 'syscall_write_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated write syscall metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.tcp_cwnd ON CLUSTER '{cluster}' +AS observoor.tcp_cwnd_local +ENGINE = Distributed('{cluster}', 'observoor', 'tcp_cwnd_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated TCP congestion window metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.tcp_retransmit ON CLUSTER '{cluster}' +AS observoor.tcp_retransmit_local +ENGINE = Distributed('{cluster}', 'observoor', 'tcp_retransmit_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated TCP retransmit metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.tcp_rtt ON CLUSTER '{cluster}' +AS observoor.tcp_rtt_local +ENGINE = Distributed('{cluster}', 'observoor', 'tcp_rtt_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated TCP round-trip time metrics from eBPF tracing of Ethereum client processes'; + +CREATE TABLE IF NOT EXISTS observoor.tcp_state_change ON CLUSTER '{cluster}' +AS observoor.tcp_state_change_local +ENGINE = Distributed('{cluster}', 'observoor', 'tcp_state_change_local', cityHash64(window_start, meta_network_name, meta_client_name)) +COMMENT 'Aggregated TCP state change events from eBPF tracing of Ethereum client processes'; + +-- admin database + +CREATE TABLE IF NOT EXISTS admin.cryo ON CLUSTER '{cluster}' +AS admin.cryo_local +ENGINE = Distributed('{cluster}', 'admin', 'cryo_local', cityHash64(dataset, mode, meta_network_name)) +COMMENT 'Tracks cryo dataset processing state per block'; + +CREATE TABLE IF NOT EXISTS admin.execution_block ON CLUSTER '{cluster}' +AS admin.execution_block_local +ENGINE = Distributed('{cluster}', 'admin', 'execution_block_local', cityHash64(block_number, processor, meta_network_name)) +COMMENT 'Tracks execution block processing state'; + +-- MATERIALIZED VIEWS + +CREATE MATERIALIZED VIEW IF NOT EXISTS default.beacon_api_slot_attestation_mv_local ON CLUSTER '{cluster}' TO default.beacon_api_slot_local +( + `slot` UInt32, + `slot_start_date_time` DateTime, + `epoch` UInt32, + `epoch_start_date_time` DateTime, + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String), + `meta_client_geo_city` LowCardinality(String), + `meta_client_geo_continent_code` LowCardinality(String), + `meta_client_geo_longitude` Nullable(Float64), + `meta_client_geo_latitude` Nullable(Float64), + `meta_consensus_implementation` LowCardinality(String), + `meta_consensus_version` LowCardinality(String), + `attestations` AggregateFunction(sum, UInt32) +) +AS SELECT + slot, + slot_start_date_time, + epoch, + epoch_start_date_time, + meta_client_name, + meta_network_name, + meta_client_geo_city, + meta_client_geo_continent_code, + meta_client_geo_longitude, + meta_client_geo_latitude, + meta_consensus_implementation, + meta_consensus_version, + sumState(toUInt32(1)) AS attestations +FROM default.beacon_api_eth_v1_events_attestation_local +GROUP BY + slot, + slot_start_date_time, + epoch, + epoch_start_date_time, + meta_client_name, + meta_network_name, + meta_client_geo_city, + meta_client_geo_continent_code, + meta_client_geo_longitude, + meta_client_geo_latitude, + meta_consensus_implementation, + meta_consensus_version; + +CREATE MATERIALIZED VIEW IF NOT EXISTS default.beacon_api_slot_block_mv_local ON CLUSTER '{cluster}' TO default.beacon_api_slot_local +( + `slot` UInt32, + `slot_start_date_time` DateTime, + `epoch` UInt32, + `epoch_start_date_time` DateTime, + `meta_client_name` LowCardinality(String), + `meta_network_name` LowCardinality(String), + `meta_client_geo_city` LowCardinality(String), + `meta_client_geo_continent_code` LowCardinality(String), + `meta_client_geo_longitude` Nullable(Float64), + `meta_client_geo_latitude` Nullable(Float64), + `meta_consensus_implementation` LowCardinality(String), + `meta_consensus_version` LowCardinality(String), + `blocks` AggregateFunction(sum, UInt16) +) +AS SELECT + slot, + slot_start_date_time, + epoch, + epoch_start_date_time, + meta_client_name, + meta_network_name, + meta_client_geo_city, + meta_client_geo_continent_code, + meta_client_geo_longitude, + meta_client_geo_latitude, + meta_consensus_implementation, + meta_consensus_version, + sumState(toUInt16(1)) AS blocks +FROM default.beacon_api_eth_v1_events_block_local +GROUP BY + slot, + slot_start_date_time, + epoch, + epoch_start_date_time, + meta_client_name, + meta_network_name, + meta_client_geo_city, + meta_client_geo_continent_code, + meta_client_geo_longitude, + meta_client_geo_latitude, + meta_consensus_implementation, + meta_consensus_version; diff --git a/deploy/migrations/clickhouse/001_initialize_schema.down.sql b/deploy/migrations/clickhouse/001_initialize_schema.down.sql deleted file mode 100644 index 407d0f39..00000000 --- a/deploy/migrations/clickhouse/001_initialize_schema.down.sql +++ /dev/null @@ -1,22 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_events_head on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_head_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_block on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_block_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_attestation on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_attestation_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_voluntary_exit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_voluntary_exit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_finalized_checkpoint on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_finalized_checkpoint_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_chain_reorg on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_chain_reorg_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_contribution_and_proof on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_contribution_and_proof_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_slot_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_slot on cluster '{cluster}' SYNC; -DROP VIEW IF EXISTS beacon_api_slot_attestation_mv_local on cluster '{cluster}' SYNC; -DROP VIEW IF EXISTS beacon_api_slot_block_mv_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS mempool_transaction on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS mempool_transaction_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/001_initialize_schema.up.sql b/deploy/migrations/clickhouse/001_initialize_schema.up.sql deleted file mode 100644 index cbf0dba8..00000000 --- a/deploy/migrations/clickhouse/001_initialize_schema.up.sql +++ /dev/null @@ -1,467 +0,0 @@ -CREATE TABLE beacon_api_eth_v1_events_head_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - block FixedString(66) Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - epoch_transition Bool, - execution_optimistic Bool, - previous_duty_dependent_root FixedString(66) Codec(ZSTD(1)), - current_duty_dependent_root FixedString(66) Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_slot (SELECT * ORDER BY slot), - PROJECTION projection_epoch (SELECT * ORDER BY epoch), - PROJECTION projection_block (SELECT * ORDER BY block) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v1_events_head on cluster '{cluster}' AS beacon_api_eth_v1_events_head_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_head_local, rand()); - -CREATE TABLE beacon_api_eth_v1_events_block_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - block FixedString(66) Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_slot (SELECT * ORDER BY slot), - PROJECTION projection_epoch (SELECT * ORDER BY epoch), - PROJECTION projection_block (SELECT * ORDER BY block) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v1_events_block on cluster '{cluster}' AS beacon_api_eth_v1_events_block_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_block_local, rand()); - -CREATE TABLE beacon_api_eth_v1_events_attestation_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - committee_index LowCardinality(String), - signature String Codec(ZSTD(1)), - aggregation_bits String Codec(ZSTD(1)), - beacon_block_root FixedString(66) Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - source_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source_root FixedString(66) Codec(ZSTD(1)), - target_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - target_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - target_root FixedString(66) Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_slot (SELECT * ORDER BY slot), - PROJECTION projection_epoch (SELECT * ORDER BY epoch), - PROJECTION projection_beacon_block_root (SELECT * ORDER BY beacon_block_root) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v1_events_attestation on cluster '{cluster}' AS beacon_api_eth_v1_events_attestation_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_attestation_local, rand()); - -CREATE TABLE beacon_api_eth_v1_events_voluntary_exit_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - validator_index UInt32 Codec(ZSTD(1)), - signature String Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_epoch (SELECT * ORDER BY epoch), - PROJECTION projection_validator_index (SELECT * ORDER BY validator_index) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v1_events_voluntary_exit on cluster '{cluster}' AS beacon_api_eth_v1_events_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_voluntary_exit_local, rand()); - -CREATE TABLE beacon_api_eth_v1_events_finalized_checkpoint_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - block FixedString(66) Codec(ZSTD(1)), - state FixedString(66) Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_epoch (SELECT * ORDER BY epoch), - PROJECTION projection_block (SELECT * ORDER BY block), - PROJECTION projection_state (SELECT * ORDER BY state) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v1_events_finalized_checkpoint on cluster '{cluster}' AS beacon_api_eth_v1_events_finalized_checkpoint_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_finalized_checkpoint_local, rand()); - -CREATE TABLE beacon_api_eth_v1_events_chain_reorg_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - depth UInt16 Codec(DoubleDelta, ZSTD(1)), - old_head_block FixedString(66) Codec(ZSTD(1)), - new_head_block FixedString(66) Codec(ZSTD(1)), - old_head_state FixedString(66) Codec(ZSTD(1)), - new_head_state FixedString(66) Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_slot (SELECT * ORDER BY slot), - PROJECTION projection_epoch (SELECT * ORDER BY epoch), - PROJECTION projection_new_head_state (SELECT * ORDER BY new_head_state), - PROJECTION projection_old_head_state (SELECT * ORDER BY old_head_state), - PROJECTION projection_new_head_block (SELECT * ORDER BY new_head_block), - PROJECTION projection_old_head_block (SELECT * ORDER BY old_head_block) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v1_events_chain_reorg on cluster '{cluster}' AS beacon_api_eth_v1_events_chain_reorg_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_chain_reorg_local, rand()); - -CREATE TABLE beacon_api_eth_v1_events_contribution_and_proof_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - aggregator_index UInt32 Codec(ZSTD(1)), - contribution_slot UInt32 Codec(DoubleDelta, ZSTD(1)), - contribution_slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - contribution_propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - contribution_beacon_block_root FixedString(66) Codec(ZSTD(1)), - contribution_subcommittee_index LowCardinality(String), - contribution_aggregation_bits String Codec(ZSTD(1)), - contribution_signature String Codec(ZSTD(1)), - contribution_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - contribution_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - selection_proof String Codec(ZSTD(1)), - signature String Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_contribution_slot (SELECT * ORDER BY contribution_slot), - PROJECTION projection_contribution_epoch (SELECT * ORDER BY contribution_epoch), - PROJECTION projection_contribution_beacon_block_root (SELECT * ORDER BY contribution_beacon_block_root) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(contribution_slot_start_date_time) -ORDER BY (contribution_slot_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v1_events_contribution_and_proof on cluster '{cluster}' AS beacon_api_eth_v1_events_contribution_and_proof_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_contribution_and_proof_local, rand()); - -CREATE TABLE beacon_api_slot_local on cluster '{cluster}' ( - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_consensus_implementation LowCardinality(String), - meta_consensus_version LowCardinality(String), - blocks AggregateFunction(sum, UInt16) Codec(ZSTD(1)), - attestations AggregateFunction(sum, UInt32) Codec(ZSTD(1)), - PROJECTION projection_slot (SELECT * ORDER BY slot), - PROJECTION projection_epoch (SELECT * ORDER BY epoch) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, slot, meta_network_name); - -CREATE MATERIALIZED VIEW beacon_api_slot_block_mv_local on cluster '{cluster}' -TO beacon_api_slot_local -AS SELECT - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - meta_client_name, - meta_network_name, - meta_client_geo_city, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_consensus_implementation, - meta_consensus_version, - sumState(toUInt16(1)) AS blocks -FROM beacon_api_eth_v1_events_block_local -GROUP BY slot, slot_start_date_time, epoch, epoch_start_date_time, meta_client_name, meta_network_name, meta_client_geo_city, meta_client_geo_continent_code, meta_client_geo_longitude, meta_client_geo_latitude, meta_consensus_implementation, meta_consensus_version; - -CREATE MATERIALIZED VIEW beacon_api_slot_attestation_mv_local on cluster '{cluster}' -TO beacon_api_slot_local -AS SELECT - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - meta_client_name, - meta_network_name, - meta_client_geo_city, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_consensus_implementation, - meta_consensus_version, - sumState(toUInt32(1)) AS attestations -FROM beacon_api_eth_v1_events_attestation_local -GROUP BY slot, slot_start_date_time, epoch, epoch_start_date_time, meta_client_name, meta_network_name, meta_client_geo_city, meta_client_geo_continent_code, meta_client_geo_longitude, meta_client_geo_latitude, meta_consensus_implementation, meta_consensus_version; - -CREATE TABLE beacon_api_slot on cluster '{cluster}' AS beacon_api_slot_local -ENGINE = Distributed('{cluster}', default, beacon_api_slot_local, rand()); - -CREATE TABLE mempool_transaction_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - hash FixedString(66) Codec(ZSTD(1)), - from FixedString(42) Codec(ZSTD(1)), - to FixedString(42) Codec(ZSTD(1)), - nonce UInt64 Codec(ZSTD(1)), - gas_price UInt128 Codec(ZSTD(1)), - gas UInt64 Codec(ZSTD(1)), - value UInt128 Codec(ZSTD(1)), - size UInt32 Codec(ZSTD(1)), - call_data_size UInt32 Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_execution_fork_id_hash LowCardinality(String), - meta_execution_fork_id_next LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_hash (SELECT * ORDER BY hash), - PROJECTION projection_from (SELECT * ORDER BY from), - PROJECTION projection_to (SELECT * ORDER BY to) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(event_date_time) -ORDER BY (event_date_time, meta_network_name, meta_client_name); - -CREATE TABLE mempool_transaction on cluster '{cluster}' AS mempool_transaction_local -ENGINE = Distributed('{cluster}', default, mempool_transaction_local, rand()); - -CREATE TABLE beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - block_root FixedString(66) Codec(ZSTD(1)), - parent_root FixedString(66) Codec(ZSTD(1)), - state_root FixedString(66) Codec(ZSTD(1)), - proposer_index UInt32 Codec(ZSTD(1)), - eth1_data_block_hash FixedString(66) Codec(ZSTD(1)), - eth1_data_deposit_root FixedString(66) Codec(ZSTD(1)), - execution_payload_block_hash FixedString(66) Codec(ZSTD(1)), - execution_payload_block_number UInt32 Codec(DoubleDelta, ZSTD(1)), - execution_payload_fee_recipient String Codec(ZSTD(1)), - execution_payload_state_root FixedString(66) Codec(ZSTD(1)), - execution_payload_parent_hash FixedString(66) Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_execution_fork_id_hash LowCardinality(String), - meta_execution_fork_id_next LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)), - PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time), - PROJECTION projection_slot (SELECT * ORDER BY slot), - PROJECTION projection_epoch (SELECT * ORDER BY epoch), - PROJECTION projection_block_root (SELECT * ORDER BY block_root), - PROJECTION projection_parent_root (SELECT * ORDER BY parent_root), - PROJECTION projection_state_root (SELECT * ORDER BY state_root), - PROJECTION projection_eth1_data_block_hash (SELECT * ORDER BY eth1_data_block_hash), - PROJECTION projection_execution_payload_block_hash (SELECT * ORDER BY execution_payload_block_hash), - PROJECTION projection_execution_payload_block_number (SELECT * ORDER BY execution_payload_block_number), - PROJECTION projection_execution_payload_state_root (SELECT * ORDER BY execution_payload_state_root), - PROJECTION projection_execution_payload_parent_hash (SELECT * ORDER BY execution_payload_parent_hash) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v2_beacon_block on cluster '{cluster}' AS beacon_api_eth_v2_beacon_block_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_local, rand()); diff --git a/deploy/migrations/clickhouse/002_attesting_validator.down.sql b/deploy/migrations/clickhouse/002_attesting_validator.down.sql deleted file mode 100644 index 57177a35..00000000 --- a/deploy/migrations/clickhouse/002_attesting_validator.down.sql +++ /dev/null @@ -1,7 +0,0 @@ -ALTER TABLE beacon_api_eth_v1_events_attestation on cluster '{cluster}' - DROP COLUMN attesting_validator_index, - DROP COLUMN attesting_validator_committee_index; - -ALTER TABLE beacon_api_eth_v1_events_attestation_local on cluster '{cluster}' - DROP COLUMN attesting_validator_index, - DROP COLUMN attesting_validator_committee_index; diff --git a/deploy/migrations/clickhouse/002_attesting_validator.up.sql b/deploy/migrations/clickhouse/002_attesting_validator.up.sql deleted file mode 100644 index 145ead96..00000000 --- a/deploy/migrations/clickhouse/002_attesting_validator.up.sql +++ /dev/null @@ -1,7 +0,0 @@ -ALTER TABLE beacon_api_eth_v1_events_attestation_local on cluster '{cluster}' - ADD COLUMN attesting_validator_index Nullable(UInt32) Codec(ZSTD(1)) AFTER committee_index, - ADD COLUMN attesting_validator_committee_index LowCardinality(String) AFTER attesting_validator_index; - -ALTER TABLE beacon_api_eth_v1_events_attestation on cluster '{cluster}' - ADD COLUMN attesting_validator_index Nullable(UInt32) Codec(ZSTD(1)) AFTER committee_index, - ADD COLUMN attesting_validator_committee_index LowCardinality(String) AFTER attesting_validator_index; diff --git a/deploy/migrations/clickhouse/003_remove_projections.down.sql b/deploy/migrations/clickhouse/003_remove_projections.down.sql deleted file mode 100644 index fbc77127..00000000 --- a/deploy/migrations/clickhouse/003_remove_projections.down.sql +++ /dev/null @@ -1,47 +0,0 @@ -ALTER TABLE beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' ADD PROJECTION projection_slot (SELECT * ORDER BY slot); -ALTER TABLE beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' ADD PROJECTION projection_epoch (SELECT * ORDER BY epoch); -ALTER TABLE beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' ADD PROJECTION projection_block (SELECT * ORDER BY block) -ALTER TABLE beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_slot (SELECT * ORDER BY slot); -ALTER TABLE beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_epoch (SELECT * ORDER BY epoch); -ALTER TABLE beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_block (SELECT * ORDER BY block); -ALTER TABLE beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' ADD PROJECTION projection_slot (SELECT * ORDER BY slot); -ALTER TABLE beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' ADD PROJECTION projection_epoch (SELECT * ORDER BY epoch); -ALTER TABLE beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' ADD PROJECTION projection_beacon_block_root (SELECT * ORDER BY beacon_block_root); -ALTER TABLE beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ADD PROJECTION projection_epoch (SELECT * ORDER BY epoch); -ALTER TABLE beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ADD PROJECTION projection_validator_index (SELECT * ORDER BY validator_index); -ALTER TABLE beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' ADD PROJECTION projection_epoch (SELECT * ORDER BY epoch); -ALTER TABLE beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' ADD PROJECTION projection_block (SELECT * ORDER BY block); -ALTER TABLE beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' ADD PROJECTION projection_state (SELECT * ORDER BY state); -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ADD PROJECTION projection_slot (SELECT * ORDER BY slot); -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ADD PROJECTION projection_epoch (SELECT * ORDER BY epoch); -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ADD PROJECTION projection_new_head_state (SELECT * ORDER BY new_head_state); -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ADD PROJECTION projection_old_head_state (SELECT * ORDER BY old_head_state); -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ADD PROJECTION projection_new_head_block (SELECT * ORDER BY new_head_block); -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ADD PROJECTION projection_old_head_block (SELECT * ORDER BY old_head_block); -ALTER TABLE beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' ADD PROJECTION projection_contribution_slot (SELECT * ORDER BY contribution_slot); -ALTER TABLE beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' ADD PROJECTION projection_contribution_epoch (SELECT * ORDER BY contribution_epoch); -ALTER TABLE beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' ADD PROJECTION projection_contribution_beacon_block_root (SELECT * ORDER BY contribution_beacon_block_root); -ALTER TABLE beacon_api_slot_local ON CLUSTER '{cluster}' ADD PROJECTION projection_slot (SELECT * ORDER BY slot); -ALTER TABLE beacon_api_slot_local ON CLUSTER '{cluster}' ADD PROJECTION projection_epoch (SELECT * ORDER BY epoch); -ALTER TABLE mempool_transaction_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE mempool_transaction_local ON CLUSTER '{cluster}' ADD PROJECTION projection_hash (SELECT * ORDER BY hash); -ALTER TABLE mempool_transaction_local ON CLUSTER '{cluster}' ADD PROJECTION projection_from (SELECT * ORDER BY from); -ALTER TABLE mempool_transaction_local ON CLUSTER '{cluster}' ADD PROJECTION projection_to (SELECT * ORDER BY to); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_event_date_time (SELECT * ORDER BY event_date_time); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_slot (SELECT * ORDER BY slot); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_epoch (SELECT * ORDER BY epoch); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_block_root (SELECT * ORDER BY block_root); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_parent_root (SELECT * ORDER BY parent_root); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_state_root (SELECT * ORDER BY state_root); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_eth1_data_block_hash (SELECT * ORDER BY eth1_data_block_hash); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_execution_payload_block_hash (SELECT * ORDER BY execution_payload_block_hash); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_execution_payload_block_number (SELECT * ORDER BY execution_payload_block_number); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_execution_payload_state_root (SELECT * ORDER BY execution_payload_state_root); -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ADD PROJECTION projection_execution_payload_parent_hash (SELECT * ORDER BY execution_payload_parent_hash); diff --git a/deploy/migrations/clickhouse/003_remove_projections.up.sql b/deploy/migrations/clickhouse/003_remove_projections.up.sql deleted file mode 100644 index 3bc74169..00000000 --- a/deploy/migrations/clickhouse/003_remove_projections.up.sql +++ /dev/null @@ -1,47 +0,0 @@ -ALTER TABLE beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' DROP PROJECTION projection_slot; -ALTER TABLE beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' DROP PROJECTION projection_epoch; -ALTER TABLE beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' DROP PROJECTION projection_block; -ALTER TABLE beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_slot; -ALTER TABLE beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_epoch; -ALTER TABLE beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_block; -ALTER TABLE beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' DROP PROJECTION projection_slot; -ALTER TABLE beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' DROP PROJECTION projection_epoch; -ALTER TABLE beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' DROP PROJECTION projection_beacon_block_root; -ALTER TABLE beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' DROP PROJECTION projection_epoch; -ALTER TABLE beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' DROP PROJECTION projection_validator_index; -ALTER TABLE beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' DROP PROJECTION projection_epoch; -ALTER TABLE beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' DROP PROJECTION projection_block; -ALTER TABLE beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' DROP PROJECTION projection_state; -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' DROP PROJECTION projection_slot; -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' DROP PROJECTION projection_epoch; -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' DROP PROJECTION projection_new_head_state; -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' DROP PROJECTION projection_old_head_state; -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' DROP PROJECTION projection_new_head_block; -ALTER TABLE beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' DROP PROJECTION projection_old_head_block; -ALTER TABLE beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' DROP PROJECTION projection_contribution_slot; -ALTER TABLE beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' DROP PROJECTION projection_contribution_epoch; -ALTER TABLE beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' DROP PROJECTION projection_contribution_beacon_block_root; -ALTER TABLE beacon_api_slot_local ON CLUSTER '{cluster}' DROP PROJECTION projection_slot; -ALTER TABLE beacon_api_slot_local ON CLUSTER '{cluster}' DROP PROJECTION projection_epoch; -ALTER TABLE mempool_transaction_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE mempool_transaction_local ON CLUSTER '{cluster}' DROP PROJECTION projection_hash; -ALTER TABLE mempool_transaction_local ON CLUSTER '{cluster}' DROP PROJECTION projection_from; -ALTER TABLE mempool_transaction_local ON CLUSTER '{cluster}' DROP PROJECTION projection_to; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_event_date_time; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_slot; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_epoch; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_block_root; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_parent_root; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_state_root; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_eth1_data_block_hash; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_execution_payload_block_hash; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_execution_payload_block_number; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_execution_payload_state_root; -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' DROP PROJECTION projection_execution_payload_parent_hash; diff --git a/deploy/migrations/clickhouse/004_dbt.down.sql b/deploy/migrations/clickhouse/004_dbt.down.sql deleted file mode 100644 index 2014046b..00000000 --- a/deploy/migrations/clickhouse/004_dbt.down.sql +++ /dev/null @@ -1 +0,0 @@ -DROP DATABASE dbt ON CLUSTER '{cluster}' diff --git a/deploy/migrations/clickhouse/004_dbt.up.sql b/deploy/migrations/clickhouse/004_dbt.up.sql deleted file mode 100644 index e229b8c3..00000000 --- a/deploy/migrations/clickhouse/004_dbt.up.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE DATABASE IF NOT EXISTS dbt ON CLUSTER '{cluster}' diff --git a/deploy/migrations/clickhouse/005_beacon_committee.down.sql b/deploy/migrations/clickhouse/005_beacon_committee.down.sql deleted file mode 100644 index 69e7d17f..00000000 --- a/deploy/migrations/clickhouse/005_beacon_committee.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_beacon_committee on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_beacon_committee_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/005_beacon_committee.up.sql b/deploy/migrations/clickhouse/005_beacon_committee.up.sql deleted file mode 100644 index 69abd636..00000000 --- a/deploy/migrations/clickhouse/005_beacon_committee.up.sql +++ /dev/null @@ -1,36 +0,0 @@ -CREATE TABLE beacon_api_eth_v1_beacon_committee_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - committee_index LowCardinality(String), - validators Array(UInt32) Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -CREATE TABLE beacon_api_eth_v1_beacon_committee on cluster '{cluster}' AS beacon_api_eth_v1_beacon_committee_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_beacon_committee_local, rand()); diff --git a/deploy/migrations/clickhouse/006_comments.down.sql b/deploy/migrations/clickhouse/006_comments.down.sql deleted file mode 100644 index a98a5ff8..00000000 --- a/deploy/migrations/clickhouse/006_comments.down.sql +++ /dev/null @@ -1,669 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN propagation_slot_start_diff '', -COMMENT COLUMN block '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN epoch_transition '', -COMMENT COLUMN execution_optimistic '', -COMMENT COLUMN previous_duty_dependent_root '', -COMMENT COLUMN current_duty_dependent_root '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN propagation_slot_start_diff '', -COMMENT COLUMN block '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN epoch_transition '', -COMMENT COLUMN execution_optimistic '', -COMMENT COLUMN previous_duty_dependent_root '', -COMMENT COLUMN current_duty_dependent_root '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN propagation_slot_start_diff '', -COMMENT COLUMN block '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN execution_optimistic '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN propagation_slot_start_diff '', -COMMENT COLUMN block '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN execution_optimistic '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_attestation ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN propagation_slot_start_diff '', -COMMENT COLUMN committee_index '', -COMMENT COLUMN attesting_validator_index '', -COMMENT COLUMN attesting_validator_committee_index '', -COMMENT COLUMN signature '', -COMMENT COLUMN aggregation_bits '', -COMMENT COLUMN beacon_block_root '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN source_epoch '', -COMMENT COLUMN source_epoch_start_date_time '', -COMMENT COLUMN source_root '', -COMMENT COLUMN target_epoch '', -COMMENT COLUMN target_epoch_start_date_time '', -COMMENT COLUMN target_root '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN propagation_slot_start_diff '', -COMMENT COLUMN committee_index '', -COMMENT COLUMN attesting_validator_index '', -COMMENT COLUMN attesting_validator_committee_index '', -COMMENT COLUMN signature '', -COMMENT COLUMN aggregation_bits '', -COMMENT COLUMN beacon_block_root '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN source_epoch '', -COMMENT COLUMN source_epoch_start_date_time '', -COMMENT COLUMN source_root '', -COMMENT COLUMN target_epoch '', -COMMENT COLUMN target_epoch_start_date_time '', -COMMENT COLUMN target_root '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN validator_index '', -COMMENT COLUMN signature '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN validator_index '', -COMMENT COLUMN signature '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN block '', -COMMENT COLUMN state '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN execution_optimistic '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN block '', -COMMENT COLUMN state '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN execution_optimistic '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN propagation_slot_start_diff '', -COMMENT COLUMN depth '', -COMMENT COLUMN old_head_block '', -COMMENT COLUMN new_head_block '', -COMMENT COLUMN old_head_state '', -COMMENT COLUMN new_head_state '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN execution_optimistic '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN propagation_slot_start_diff '', -COMMENT COLUMN depth '', -COMMENT COLUMN old_head_block '', -COMMENT COLUMN new_head_block '', -COMMENT COLUMN old_head_state '', -COMMENT COLUMN new_head_state '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN execution_optimistic '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN aggregator_index '', -COMMENT COLUMN contribution_slot '', -COMMENT COLUMN contribution_slot_start_date_time '', -COMMENT COLUMN contribution_propagation_slot_start_diff '', -COMMENT COLUMN contribution_beacon_block_root '', -COMMENT COLUMN contribution_subcommittee_index '', -COMMENT COLUMN contribution_aggregation_bits '', -COMMENT COLUMN contribution_signature '', -COMMENT COLUMN contribution_epoch '', -COMMENT COLUMN contribution_epoch_start_date_time '', -COMMENT COLUMN selection_proof '', -COMMENT COLUMN signature '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN aggregator_index '', -COMMENT COLUMN contribution_slot '', -COMMENT COLUMN contribution_slot_start_date_time '', -COMMENT COLUMN contribution_propagation_slot_start_diff '', -COMMENT COLUMN contribution_beacon_block_root '', -COMMENT COLUMN contribution_subcommittee_index '', -COMMENT COLUMN contribution_aggregation_bits '', -COMMENT COLUMN contribution_signature '', -COMMENT COLUMN contribution_epoch '', -COMMENT COLUMN contribution_epoch_start_date_time '', -COMMENT COLUMN selection_proof '', -COMMENT COLUMN signature '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_slot ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN blocks'', -COMMENT COLUMN attestations''; - -ALTER TABLE default.beacon_api_slot_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN blocks'', -COMMENT COLUMN attestations''; - -ALTER TABLE default.mempool_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN hash '', -COMMENT COLUMN from'', -COMMENT COLUMN to'', -COMMENT COLUMN nonce '', -COMMENT COLUMN gas_price '', -COMMENT COLUMN gas '', -COMMENT COLUMN value '', -COMMENT COLUMN size '', -COMMENT COLUMN call_data_size '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_execution_fork_id_hash '', -COMMENT COLUMN meta_execution_fork_id_next '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.mempool_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN hash '', -COMMENT COLUMN from'', -COMMENT COLUMN to'', -COMMENT COLUMN nonce '', -COMMENT COLUMN gas_price '', -COMMENT COLUMN gas '', -COMMENT COLUMN value '', -COMMENT COLUMN size '', -COMMENT COLUMN call_data_size '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_execution_fork_id_hash '', -COMMENT COLUMN meta_execution_fork_id_next '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN block_root '', -COMMENT COLUMN parent_root '', -COMMENT COLUMN state_root '', -COMMENT COLUMN proposer_index '', -COMMENT COLUMN eth1_data_block_hash '', -COMMENT COLUMN eth1_data_deposit_root '', -COMMENT COLUMN execution_payload_block_hash '', -COMMENT COLUMN execution_payload_block_number '', -COMMENT COLUMN execution_payload_fee_recipient '', -COMMENT COLUMN execution_payload_state_root '', -COMMENT COLUMN execution_payload_parent_hash '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_execution_fork_id_hash '', -COMMENT COLUMN meta_execution_fork_id_next '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN block_root '', -COMMENT COLUMN parent_root '', -COMMENT COLUMN state_root '', -COMMENT COLUMN proposer_index '', -COMMENT COLUMN eth1_data_block_hash '', -COMMENT COLUMN eth1_data_deposit_root '', -COMMENT COLUMN execution_payload_block_hash '', -COMMENT COLUMN execution_payload_block_number '', -COMMENT COLUMN execution_payload_fee_recipient '', -COMMENT COLUMN execution_payload_state_root '', -COMMENT COLUMN execution_payload_parent_hash '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_execution_fork_id_hash '', -COMMENT COLUMN meta_execution_fork_id_next '', -COMMENT COLUMN meta_labels ''; diff --git a/deploy/migrations/clickhouse/006_comments.up.sql b/deploy/migrations/clickhouse/006_comments.up.sql deleted file mode 100644 index d9a999e0..00000000 --- a/deploy/migrations/clickhouse/006_comments.up.sql +++ /dev/null @@ -1,669 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN block 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN epoch_transition 'If the event is an epoch transition', -COMMENT COLUMN execution_optimistic 'If the attached beacon node is running in execution optimistic mode', -COMMENT COLUMN previous_duty_dependent_root 'The previous duty dependent root in the beacon API event stream payload', -COMMENT COLUMN current_duty_dependent_root 'The current duty dependent root in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN block 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN epoch_transition 'If the event is an epoch transition', -COMMENT COLUMN execution_optimistic 'If the attached beacon node is running in execution optimistic mode', -COMMENT COLUMN previous_duty_dependent_root 'The previous duty dependent root in the beacon API event stream payload', -COMMENT COLUMN current_duty_dependent_root 'The current duty dependent root in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN block 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'If the attached beacon node is running in execution optimistic mode', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN block 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'If the attached beacon node is running in execution optimistic mode', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_attestation ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "attestation" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN committee_index 'The committee index in the beacon API event stream payload', -COMMENT COLUMN attesting_validator_index 'The index of the validator attesting to the event', -COMMENT COLUMN attesting_validator_committee_index 'The committee index of the attesting validator', -COMMENT COLUMN signature 'The signature of the event in the beacon API event stream payload', -COMMENT COLUMN aggregation_bits 'The aggregation bits of the event in the beacon API event stream payload', -COMMENT COLUMN beacon_block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN source_epoch 'The source epoch number in the beacon API event stream payload', -COMMENT COLUMN source_epoch_start_date_time 'The wall clock time when the source epoch started', -COMMENT COLUMN source_root 'The source beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN target_epoch 'The target epoch number in the beacon API event stream payload', -COMMENT COLUMN target_epoch_start_date_time 'The wall clock time when the target epoch started', -COMMENT COLUMN target_root 'The target beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "attestation" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN committee_index 'The committee index in the beacon API event stream payload', -COMMENT COLUMN attesting_validator_index 'The index of the validator attesting to the event', -COMMENT COLUMN attesting_validator_committee_index 'The committee index of the attesting validator', -COMMENT COLUMN signature 'The signature of the event in the beacon API event stream payload', -COMMENT COLUMN aggregation_bits 'The aggregation bits of the event in the beacon API event stream payload', -COMMENT COLUMN beacon_block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN source_epoch 'The source epoch number in the beacon API event stream payload', -COMMENT COLUMN source_epoch_start_date_time 'The wall clock time when the source epoch started', -COMMENT COLUMN source_root 'The source beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN target_epoch 'The target epoch number in the beacon API event stream payload', -COMMENT COLUMN target_epoch_start_date_time 'The wall clock time when the target epoch started', -COMMENT COLUMN target_root 'The target beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN validator_index 'The index of the validator making the voluntary exit', -COMMENT COLUMN signature 'The signature of the voluntary exit in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN validator_index 'The index of the validator making the voluntary exit', -COMMENT COLUMN signature 'The signature of the voluntary exit in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN block 'The finalized block root hash in the beacon API event stream payload', -COMMENT COLUMN state 'The finalized state root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'Whether the execution of the epoch was optimistic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN block 'The finalized block root hash in the beacon API event stream payload', -COMMENT COLUMN state 'The finalized state root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'Whether the execution of the epoch was optimistic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'The slot number of the chain reorg event in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the reorg slot started', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slots between when the reorg occurred and when the sentry received the event', -COMMENT COLUMN depth 'The depth of the chain reorg in the beacon API event stream payload', -COMMENT COLUMN old_head_block 'The old head block root hash in the beacon API event stream payload', -COMMENT COLUMN new_head_block 'The new head block root hash in the beacon API event stream payload', -COMMENT COLUMN old_head_state 'The old head state root hash in the beacon API event stream payload', -COMMENT COLUMN new_head_state 'The new head state root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'Whether the execution of the epoch was optimistic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'The slot number of the chain reorg event in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the reorg slot started', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slots between when the reorg occurred and when the sentry received the event', -COMMENT COLUMN depth 'The depth of the chain reorg in the beacon API event stream payload', -COMMENT COLUMN old_head_block 'The old head block root hash in the beacon API event stream payload', -COMMENT COLUMN new_head_block 'The new head block root hash in the beacon API event stream payload', -COMMENT COLUMN old_head_state 'The old head state root hash in the beacon API event stream payload', -COMMENT COLUMN new_head_state 'The new head state root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'Whether the execution of the epoch was optimistic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN aggregator_index 'The validator index of the aggregator in the beacon API event stream payload', -COMMENT COLUMN contribution_slot 'The slot number of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_slot_start_date_time 'The wall clock time when the contribution slot started', -COMMENT COLUMN contribution_propagation_slot_start_diff 'Difference in slots between when the contribution occurred and when the sentry received the event', -COMMENT COLUMN contribution_beacon_block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN contribution_subcommittee_index 'The subcommittee index of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_aggregation_bits 'The aggregation bits of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_signature 'The signature of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_epoch 'The epoch number of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_epoch_start_date_time 'The wall clock time when the contribution epoch started', -COMMENT COLUMN selection_proof 'The selection proof in the beacon API event stream payload', -COMMENT COLUMN signature 'The signature in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN aggregator_index 'The validator index of the aggregator in the beacon API event stream payload', -COMMENT COLUMN contribution_slot 'The slot number of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_slot_start_date_time 'The wall clock time when the contribution slot started', -COMMENT COLUMN contribution_propagation_slot_start_diff 'Difference in slots between when the contribution occurred and when the sentry received the event', -COMMENT COLUMN contribution_beacon_block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN contribution_subcommittee_index 'The subcommittee index of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_aggregation_bits 'The aggregation bits of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_signature 'The signature of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_epoch 'The epoch number of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_epoch_start_date_time 'The wall clock time when the contribution epoch started', -COMMENT COLUMN selection_proof 'The selection proof in the beacon API event stream payload', -COMMENT COLUMN signature 'The signature in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_slot ON CLUSTER '{cluster}' -MODIFY COMMENT 'Aggregated beacon API slot data. Each row represents a slot from each sentry client attached to a beacon node.', -COMMENT COLUMN slot 'Slot number', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'Epoch number', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN blocks 'The number of beacon blocks seen in the slot', -COMMENT COLUMN attestations 'The number of attestations seen in the slot'; - -ALTER TABLE default.beacon_api_slot_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Aggregated beacon API slot data. Each row represents a slot from each sentry client attached to a beacon node.', -COMMENT COLUMN slot 'Slot number', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'Epoch number', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN blocks 'The number of beacon blocks seen in the slot', -COMMENT COLUMN attestations 'The number of attestations seen in the slot'; - -ALTER TABLE default.mempool_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT 'Each row represents a transaction that was seen in the mempool by a sentry client. Sentries can report the same transaction multiple times if it has been long enough since the last report.', -COMMENT COLUMN event_date_time 'The time when the sentry saw the transaction in the mempool', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN size 'The size of the transaction data in bytes', -COMMENT COLUMN call_data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_execution_fork_id_hash 'The hash of the fork ID of the current Ethereum network', -COMMENT COLUMN meta_execution_fork_id_next 'The fork ID of the next planned Ethereum network upgrade', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.mempool_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Each row represents a transaction that was seen in the mempool by a sentry client. Sentries can report the same transaction multiple times if it has been long enough since the last report.', -COMMENT COLUMN event_date_time 'The time when the sentry saw the transaction in the mempool', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN size 'The size of the transaction data in bytes', -COMMENT COLUMN call_data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_execution_fork_id_hash 'The hash of the fork ID of the current Ethereum network', -COMMENT COLUMN meta_execution_fork_id_next 'The fork ID of the next planned Ethereum network upgrade', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the reorg slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN parent_root 'The root hash of the parent beacon block', -COMMENT COLUMN state_root 'The root hash of the beacon state at this block', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN eth1_data_block_hash 'The block hash of the associated execution block', -COMMENT COLUMN eth1_data_deposit_root 'The root of the deposit tree in the associated execution block', -COMMENT COLUMN execution_payload_block_hash 'The block hash of the execution payload', -COMMENT COLUMN execution_payload_block_number 'The block number of the execution payload', -COMMENT COLUMN execution_payload_fee_recipient 'The recipient of the fee for this execution payload', -COMMENT COLUMN execution_payload_state_root 'The state root of the execution payload', -COMMENT COLUMN execution_payload_parent_hash 'The parent hash of the execution payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_execution_fork_id_hash 'The hash of the fork ID of the current Ethereum network', -COMMENT COLUMN meta_execution_fork_id_next 'The fork ID of the next planned Ethereum network upgrade', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the reorg slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN parent_root 'The root hash of the parent beacon block', -COMMENT COLUMN state_root 'The root hash of the beacon state at this block', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN eth1_data_block_hash 'The block hash of the associated execution block', -COMMENT COLUMN eth1_data_deposit_root 'The root of the deposit tree in the associated execution block', -COMMENT COLUMN execution_payload_block_hash 'The block hash of the execution payload', -COMMENT COLUMN execution_payload_block_number 'The block number of the execution payload', -COMMENT COLUMN execution_payload_fee_recipient 'The recipient of the fee for this execution payload', -COMMENT COLUMN execution_payload_state_root 'The state root of the execution payload', -COMMENT COLUMN execution_payload_parent_hash 'The parent hash of the execution payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_execution_fork_id_hash 'The hash of the fork ID of the current Ethereum network', -COMMENT COLUMN meta_execution_fork_id_next 'The fork ID of the next planned Ethereum network upgrade', -COMMENT COLUMN meta_labels 'Labels associated with the event'; diff --git a/deploy/migrations/clickhouse/007_tmp.down.sql b/deploy/migrations/clickhouse/007_tmp.down.sql deleted file mode 100644 index a78a4b18..00000000 --- a/deploy/migrations/clickhouse/007_tmp.down.sql +++ /dev/null @@ -1 +0,0 @@ -DROP DATABASE tmp ON CLUSTER '{cluster}' diff --git a/deploy/migrations/clickhouse/007_tmp.up.sql b/deploy/migrations/clickhouse/007_tmp.up.sql deleted file mode 100644 index 757da6cd..00000000 --- a/deploy/migrations/clickhouse/007_tmp.up.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE DATABASE IF NOT EXISTS tmp ON CLUSTER '{cluster}' diff --git a/deploy/migrations/clickhouse/008_block_transactions_size.down.sql b/deploy/migrations/clickhouse/008_block_transactions_size.down.sql deleted file mode 100644 index 3d27976b..00000000 --- a/deploy/migrations/clickhouse/008_block_transactions_size.down.sql +++ /dev/null @@ -1,7 +0,0 @@ -ALTER TABLE beacon_api_eth_v2_beacon_block on cluster '{cluster}' - DROP COLUMN execution_payload_transactions_count, - DROP COLUMN execution_payload_transactions_total_bytes; - -ALTER TABLE beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' - DROP COLUMN execution_payload_transactions_count, - DROP COLUMN execution_payload_transactions_total_bytes; diff --git a/deploy/migrations/clickhouse/008_block_transactions_size.up.sql b/deploy/migrations/clickhouse/008_block_transactions_size.up.sql deleted file mode 100644 index f58cd597..00000000 --- a/deploy/migrations/clickhouse/008_block_transactions_size.up.sql +++ /dev/null @@ -1,7 +0,0 @@ -ALTER TABLE beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' - ADD COLUMN execution_payload_transactions_count Nullable(UInt32) Codec(ZSTD(1)) AFTER execution_payload_parent_hash, - ADD COLUMN execution_payload_transactions_total_bytes Nullable(UInt32) Codec(ZSTD(1)) AFTER execution_payload_transactions_count; - -ALTER TABLE beacon_api_eth_v2_beacon_block on cluster '{cluster}' - ADD COLUMN execution_payload_transactions_count Nullable(UInt32) Codec(ZSTD(1)) AFTER execution_payload_parent_hash, - ADD COLUMN execution_payload_transactions_total_bytes Nullable(UInt32) Codec(ZSTD(1)) AFTER execution_payload_transactions_count; diff --git a/deploy/migrations/clickhouse/009_beacon_block_v2_update.down.sql b/deploy/migrations/clickhouse/009_beacon_block_v2_update.down.sql deleted file mode 100644 index e70eb21b..00000000 --- a/deploy/migrations/clickhouse/009_beacon_block_v2_update.down.sql +++ /dev/null @@ -1,21 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS meta_consensus_version, - DROP COLUMN IF EXISTS meta_consensus_version_major, - DROP COLUMN IF EXISTS meta_consensus_version_minor, - DROP COLUMN IF EXISTS meta_consensus_version_patch, - DROP COLUMN IF EXISTS meta_consensus_implementation, - ADD COLUMN IF NOT EXISTS meta_execution_fork_id_hash LowCardinality(String) AFTER meta_network_name, - ADD COLUMN IF NOT EXISTS meta_execution_fork_id_next LowCardinality(String) AFTER meta_execution_fork_id_hash, - COMMENT COLUMN execution_payload_transactions_count '', - COMMENT COLUMN execution_payload_transactions_total_bytes ''; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS meta_consensus_version, - DROP COLUMN IF EXISTS meta_consensus_version_major, - DROP COLUMN IF EXISTS meta_consensus_version_minor, - DROP COLUMN IF EXISTS meta_consensus_version_patch, - DROP COLUMN IF EXISTS meta_consensus_implementation, - ADD COLUMN IF NOT EXISTS meta_execution_fork_id_hash LowCardinality(String) AFTER meta_network_name, - ADD COLUMN IF NOT EXISTS meta_execution_fork_id_next LowCardinality(String) AFTER meta_execution_fork_id_hash, - COMMENT COLUMN execution_payload_transactions_count '', - COMMENT COLUMN execution_payload_transactions_total_bytes ''; diff --git a/deploy/migrations/clickhouse/009_beacon_block_v2_update.up.sql b/deploy/migrations/clickhouse/009_beacon_block_v2_update.up.sql deleted file mode 100644 index c795d3ca..00000000 --- a/deploy/migrations/clickhouse/009_beacon_block_v2_update.up.sql +++ /dev/null @@ -1,31 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' - ADD COLUMN meta_consensus_version LowCardinality(String) AFTER meta_network_name, - ADD COLUMN meta_consensus_version_major LowCardinality(String) AFTER meta_consensus_version, - ADD COLUMN meta_consensus_version_minor LowCardinality(String) AFTER meta_consensus_version_major, - ADD COLUMN meta_consensus_version_patch LowCardinality(String) AFTER meta_consensus_version_minor, - ADD COLUMN meta_consensus_implementation LowCardinality(String) AFTER meta_consensus_version_patch, - DROP COLUMN meta_execution_fork_id_hash, - DROP COLUMN meta_execution_fork_id_next, - COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', - COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', - COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', - COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', - COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', - COMMENT COLUMN execution_payload_transactions_count 'The transaction count of the execution payload', - COMMENT COLUMN execution_payload_transactions_total_bytes 'The transaction total bytes of the execution payload'; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' - ADD COLUMN meta_consensus_version LowCardinality(String) AFTER meta_network_name, - ADD COLUMN meta_consensus_version_major LowCardinality(String) AFTER meta_consensus_version, - ADD COLUMN meta_consensus_version_minor LowCardinality(String) AFTER meta_consensus_version_major, - ADD COLUMN meta_consensus_version_patch LowCardinality(String) AFTER meta_consensus_version_minor, - ADD COLUMN meta_consensus_implementation LowCardinality(String) AFTER meta_consensus_version_patch, - DROP COLUMN meta_execution_fork_id_hash, - DROP COLUMN meta_execution_fork_id_next, - COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', - COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', - COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', - COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', - COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', - COMMENT COLUMN execution_payload_transactions_count 'The transaction count of the execution payload', - COMMENT COLUMN execution_payload_transactions_total_bytes 'The transaction total bytes of the execution payload'; diff --git a/deploy/migrations/clickhouse/010_validator_attestation_data.down.sql b/deploy/migrations/clickhouse/010_validator_attestation_data.down.sql deleted file mode 100644 index 98ad7703..00000000 --- a/deploy/migrations/clickhouse/010_validator_attestation_data.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_validator_attestation_data on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_validator_attestation_data_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/010_validator_attestation_data.up.sql b/deploy/migrations/clickhouse/010_validator_attestation_data.up.sql deleted file mode 100644 index 69638da3..00000000 --- a/deploy/migrations/clickhouse/010_validator_attestation_data.up.sql +++ /dev/null @@ -1,86 +0,0 @@ -CREATE TABLE beacon_api_eth_v1_validator_attestation_data_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - committee_index LowCardinality(String), - beacon_block_root FixedString(66) Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - source_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source_root FixedString(66) Codec(ZSTD(1)), - target_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - target_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - target_root FixedString(66) Codec(ZSTD(1)), - request_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - request_duration UInt32 Codec(ZSTD(1)), - request_slot_start_diff UInt32 Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) Codec(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API validator attestation data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API validator attestation data payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN committee_index 'The committee index in the beacon API validator attestation data payload', -COMMENT COLUMN beacon_block_root 'The beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN source_epoch 'The source epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN source_epoch_start_date_time 'The wall clock time when the source epoch started', -COMMENT COLUMN source_root 'The source beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN target_epoch 'The target epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN target_epoch_start_date_time 'The wall clock time when the target epoch started', -COMMENT COLUMN target_root 'The target beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN request_date_time 'When the request was sent to the beacon node', -COMMENT COLUMN request_duration 'The request duration in milliseconds', -COMMENT COLUMN request_slot_start_diff 'The difference between the request_date_time and the slot_start_date_time', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v1_validator_attestation_data on cluster '{cluster}' AS beacon_api_eth_v1_validator_attestation_data_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_validator_attestation_data_local, rand()); diff --git a/deploy/migrations/clickhouse/011_block_transactions_size.down.sql b/deploy/migrations/clickhouse/011_block_transactions_size.down.sql deleted file mode 100644 index 791e30f0..00000000 --- a/deploy/migrations/clickhouse/011_block_transactions_size.down.sql +++ /dev/null @@ -1,9 +0,0 @@ -ALTER TABLE beacon_api_eth_v2_beacon_block on cluster '{cluster}' - DROP COLUMN block_total_bytes, - DROP COLUMN block_total_bytes_compressed, - DROP COLUMN execution_payload_transactions_total_bytes_compressed; - -ALTER TABLE beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' - DROP COLUMN block_total_bytes, - DROP COLUMN block_total_bytes_compressed, - DROP COLUMN execution_payload_transactions_total_bytes_compressed; diff --git a/deploy/migrations/clickhouse/011_block_transactions_size.up.sql b/deploy/migrations/clickhouse/011_block_transactions_size.up.sql deleted file mode 100644 index 647d3753..00000000 --- a/deploy/migrations/clickhouse/011_block_transactions_size.up.sql +++ /dev/null @@ -1,15 +0,0 @@ -ALTER TABLE beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' - ADD COLUMN block_total_bytes Nullable(UInt32) Codec(ZSTD(1)) AFTER block_root, - ADD COLUMN block_total_bytes_compressed Nullable(UInt32) Codec(ZSTD(1)) AFTER block_total_bytes, - ADD COLUMN execution_payload_transactions_total_bytes_compressed Nullable(UInt32) Codec(ZSTD(1)) AFTER execution_payload_transactions_total_bytes, - COMMENT COLUMN block_total_bytes 'The total bytes of the beacon block payload', - COMMENT COLUMN block_total_bytes_compressed 'The total bytes of the beacon block payload when compressed using snappy', - COMMENT COLUMN execution_payload_transactions_total_bytes_compressed 'The transaction total bytes of the execution payload when compressed using snappy'; - -ALTER TABLE beacon_api_eth_v2_beacon_block on cluster '{cluster}' - ADD COLUMN block_total_bytes Nullable(UInt32) Codec(ZSTD(1)) AFTER block_root, - ADD COLUMN block_total_bytes_compressed Nullable(UInt32) Codec(ZSTD(1)) AFTER block_total_bytes, - ADD COLUMN execution_payload_transactions_total_bytes_compressed Nullable(UInt32) Codec(ZSTD(1)) AFTER execution_payload_transactions_total_bytes, - COMMENT COLUMN block_total_bytes 'The total bytes of the beacon block payload', - COMMENT COLUMN block_total_bytes_compressed 'The total bytes of the beacon block payload when compressed using snappy', - COMMENT COLUMN execution_payload_transactions_total_bytes_compressed 'The transaction total bytes of the execution payload when compressed using snappy'; diff --git a/deploy/migrations/clickhouse/012_cannon.down.sql b/deploy/migrations/clickhouse/012_cannon.down.sql deleted file mode 100644 index 8cfcbae7..00000000 --- a/deploy/migrations/clickhouse/012_cannon.down.sql +++ /dev/null @@ -1,12 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_proposer_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_proposer_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_attester_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_attester_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_bls_to_execution_change on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_bls_to_execution_change_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_execution_transaction on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_execution_transaction_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_voluntary_exit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_voluntary_exit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_deposit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_deposit_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/012_cannon.up.sql b/deploy/migrations/clickhouse/012_cannon.up.sql deleted file mode 100644 index 637b74d9..00000000 --- a/deploy/migrations/clickhouse/012_cannon.up.sql +++ /dev/null @@ -1,525 +0,0 @@ -CREATE TABLE default.beacon_api_eth_v2_beacon_block_proposer_slashing_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - signed_header_1_message_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_1_message_proposer_index UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_1_message_body_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_message_parent_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_message_state_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_signature String CODEC(ZSTD(1)), - signed_header_2_message_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_2_message_proposer_index UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_2_message_body_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_message_parent_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_message_state_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains proposer slashing from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN signed_header_1_message_slot 'The slot number from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_proposer_index 'The proposer index from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_body_root 'The body root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_parent_root 'The parent root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_state_root 'The state root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_signature 'The signature for the first signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_slot 'The slot number from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_proposer_index 'The proposer index from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_body_root 'The body root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_parent_root 'The parent root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_state_root 'The state root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_signature 'The signature for the second signed header in the slashing payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v2_beacon_block_proposer_slashing on cluster '{cluster}' AS beacon_api_eth_v2_beacon_block_proposer_slashing_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_proposer_slashing_local, rand()); - -CREATE TABLE default.beacon_api_eth_v2_beacon_block_attester_slashing_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - attestation_1_attesting_indices Array(UInt32) Codec(ZSTD(1)), - attestation_1_signature String CODEC(ZSTD(1)), - attestation_1_data_beacon_block_root FixedString(66) CODEC(ZSTD(1)), - attestation_1_data_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_index UInt32 Codec(ZSTD(1)), - attestation_1_data_source_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_source_root FixedString(66) CODEC(ZSTD(1)), - attestation_1_data_target_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_target_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_attesting_indices Array(UInt32) Codec(ZSTD(1)), - attestation_2_signature String CODEC(ZSTD(1)), - attestation_2_data_beacon_block_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_data_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_index UInt32 Codec(ZSTD(1)), - attestation_2_data_source_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_source_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_data_target_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_target_root FixedString(66) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains attester slashing from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN attestation_1_attesting_indices 'The attesting indices from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_signature 'The signature from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_beacon_block_root 'The beacon block root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_slot 'The slot number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_index 'The attestor index from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_source_epoch 'The source epoch number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_source_root 'The source root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_target_epoch 'The target epoch number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_target_root 'The target root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_2_attesting_indices 'The attesting indices from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_signature 'The signature from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_beacon_block_root 'The beacon block root from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_slot 'The slot number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_index 'The attestor index from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_source_epoch 'The source epoch number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_source_root 'The source root from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_target_epoch 'The target epoch number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_target_root 'The target root from the second attestation in the slashing payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v2_beacon_block_attester_slashing on cluster '{cluster}' AS beacon_api_eth_v2_beacon_block_attester_slashing_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_attester_slashing_local, rand()); - -CREATE TABLE default.beacon_api_eth_v2_beacon_block_bls_to_execution_change_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - exchanging_message_validator_index UInt32 CODEC(ZSTD(1)), - exchanging_message_from_bls_pubkey String CODEC(ZSTD(1)), - exchanging_message_to_execution_address FixedString(42) CODEC(ZSTD(1)), - exchanging_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains bls to execution change from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN exchanging_message_validator_index 'The validator index from the exchanging message', -COMMENT COLUMN exchanging_message_from_bls_pubkey 'The BLS public key from the exchanging message', -COMMENT COLUMN exchanging_message_to_execution_address 'The execution address from the exchanging message', -COMMENT COLUMN exchanging_signature 'The signature for the exchanging message', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v2_beacon_block_bls_to_execution_change on cluster '{cluster}' AS beacon_api_eth_v2_beacon_block_bls_to_execution_change_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_bls_to_execution_change_local, rand()); - -CREATE TABLE default.beacon_api_eth_v2_beacon_block_execution_transaction_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - position UInt32 CODEC(DoubleDelta, ZSTD(1)), - hash FixedString(66) Codec(ZSTD(1)), - from FixedString(42) Codec(ZSTD(1)), - to Nullable(FixedString(42)) Codec(ZSTD(1)), - nonce UInt64 Codec(ZSTD(1)), - gas_price UInt128 Codec(ZSTD(1)), - gas UInt64 Codec(ZSTD(1)), - value UInt128 Codec(ZSTD(1)), - type UInt8 Codec(ZSTD(1)), - size UInt32 Codec(ZSTD(1)), - call_data_size UInt32 Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution transaction from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN position 'The position of the transaction in the beacon block', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN type 'The type of the transaction', -COMMENT COLUMN size 'The size of the transaction data in bytes', -COMMENT COLUMN call_data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v2_beacon_block_execution_transaction on cluster '{cluster}' AS beacon_api_eth_v2_beacon_block_execution_transaction_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_execution_transaction_local, rand()); - -CREATE TABLE default.beacon_api_eth_v2_beacon_block_voluntary_exit_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - voluntary_exit_message_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - voluntary_exit_message_validator_index UInt32 CODEC(ZSTD(1)), - voluntary_exit_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a voluntary exit from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN voluntary_exit_message_epoch 'The epoch number from the exit message', -COMMENT COLUMN voluntary_exit_message_validator_index 'The validator index from the exit message', -COMMENT COLUMN voluntary_exit_signature 'The signature of the exit message', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v2_beacon_block_voluntary_exit on cluster '{cluster}' AS beacon_api_eth_v2_beacon_block_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_voluntary_exit_local, rand()); - -CREATE TABLE default.beacon_api_eth_v2_beacon_block_deposit_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - deposit_proof Array(String) CODEC(ZSTD(1)), - deposit_data_pubkey String CODEC(ZSTD(1)), - deposit_data_withdrawal_credentials FixedString(66) CODEC(ZSTD(1)), - deposit_data_amount UInt128 CODEC(ZSTD(1)), - deposit_data_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_deposit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a deposit from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN deposit_proof 'The proof of the deposit data', -COMMENT COLUMN deposit_data_pubkey 'The BLS public key of the validator from the deposit data', -COMMENT COLUMN deposit_data_withdrawal_credentials 'The withdrawal credentials of the validator from the deposit data', -COMMENT COLUMN deposit_data_amount 'The amount of the deposit from the deposit data', -COMMENT COLUMN deposit_data_signature 'The signature of the deposit data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v2_beacon_block_deposit on cluster '{cluster}' AS beacon_api_eth_v2_beacon_block_deposit_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_deposit_local, rand()); diff --git a/deploy/migrations/clickhouse/013_mempool_txn_type.down.sql b/deploy/migrations/clickhouse/013_mempool_txn_type.down.sql deleted file mode 100644 index 25895c3d..00000000 --- a/deploy/migrations/clickhouse/013_mempool_txn_type.down.sql +++ /dev/null @@ -1,11 +0,0 @@ -ALTER TABLE mempool_transaction_local on cluster '{cluster}' - DROP COLUMN type; - -ALTER TABLE mempool_transaction on cluster '{cluster}' - DROP COLUMN type; - -ALTER TABLE mempool_transaction_local on cluster '{cluster}' -MODIFY COLUMN to FixedString(42) Codec(ZSTD(1)); - -ALTER TABLE mempool_transaction on cluster '{cluster}' -MODIFY COLUMN to FixedString(42) Codec(ZSTD(1)); diff --git a/deploy/migrations/clickhouse/013_mempool_txn_type.up.sql b/deploy/migrations/clickhouse/013_mempool_txn_type.up.sql deleted file mode 100644 index 8e370b6d..00000000 --- a/deploy/migrations/clickhouse/013_mempool_txn_type.up.sql +++ /dev/null @@ -1,17 +0,0 @@ -ALTER TABLE mempool_transaction_local on cluster '{cluster}' -MODIFY COLUMN to Nullable(FixedString(42)) Codec(ZSTD(1)); - -ALTER TABLE mempool_transaction on cluster '{cluster}' -MODIFY COLUMN to Nullable(FixedString(42)) Codec(ZSTD(1)); - -ALTER TABLE mempool_transaction_local on cluster '{cluster}' -ADD COLUMN type Nullable(UInt8) Codec(ZSTD(1)) AFTER value; - -ALTER TABLE mempool_transaction on cluster '{cluster}' -ADD COLUMN type Nullable(UInt8) Codec(ZSTD(1)) AFTER value; - -ALTER TABLE mempool_transaction_local on cluster '{cluster}' -COMMENT COLUMN type 'The type of the transaction'; - -ALTER TABLE mempool_transaction on cluster '{cluster}' -COMMENT COLUMN type 'The type of the transaction'; diff --git a/deploy/migrations/clickhouse/014_cannon_rename.down.sql b/deploy/migrations/clickhouse/014_cannon_rename.down.sql deleted file mode 100644 index 4dca5fbe..00000000 --- a/deploy/migrations/clickhouse/014_cannon_rename.down.sql +++ /dev/null @@ -1,22 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_block on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_proposer_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_proposer_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_attester_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_attester_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_bls_to_execution_change_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_execution_transaction on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_execution_transaction_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_voluntary_exit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_voluntary_exit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_deposit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_deposit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_withdrawal on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_withdrawal_local on cluster '{cluster}' SYNC; - -ALTER TABLE beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' - DROP COLUMN block_version; - -ALTER TABLE beacon_api_eth_v2_beacon_block on cluster '{cluster}' - DROP COLUMN block_version; diff --git a/deploy/migrations/clickhouse/014_cannon_rename.up.sql b/deploy/migrations/clickhouse/014_cannon_rename.up.sql deleted file mode 100644 index 412e1276..00000000 --- a/deploy/migrations/clickhouse/014_cannon_rename.up.sql +++ /dev/null @@ -1,728 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_proposer_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_proposer_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_attester_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_attester_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_bls_to_execution_change on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_bls_to_execution_change_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_execution_transaction on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_execution_transaction_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_voluntary_exit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_voluntary_exit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_deposit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v2_beacon_block_deposit_local on cluster '{cluster}' SYNC; - -ALTER TABLE beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' -ADD COLUMN block_version LowCardinality(String) Codec(ZSTD(1)) AFTER block_root; - -ALTER TABLE beacon_api_eth_v2_beacon_block on cluster '{cluster}' -ADD COLUMN block_version LowCardinality(String) Codec(ZSTD(1)) AFTER block_root; - -ALTER TABLE beacon_api_eth_v2_beacon_block_local on cluster '{cluster}' -COMMENT COLUMN block_version 'The version of the beacon block'; - -ALTER TABLE beacon_api_eth_v2_beacon_block on cluster '{cluster}' -COMMENT COLUMN block_version 'The version of the beacon block'; - -CREATE TABLE default.canonical_beacon_block_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - block_total_bytes Nullable(UInt32) CODEC(ZSTD(1)), - block_total_bytes_compressed Nullable(UInt32) CODEC(ZSTD(1)), - parent_root FixedString(66) CODEC(ZSTD(1)), - state_root FixedString(66) CODEC(ZSTD(1)), - proposer_index UInt32 CODEC(ZSTD(1)), - eth1_data_block_hash FixedString(66) CODEC(ZSTD(1)), - eth1_data_deposit_root FixedString(66) CODEC(ZSTD(1)), - execution_payload_block_hash FixedString(66) CODEC(ZSTD(1)), - execution_payload_block_number UInt32 CODEC(DoubleDelta, ZSTD(1)), - execution_payload_fee_recipient String CODEC(ZSTD(1)), - execution_payload_state_root FixedString(66) CODEC(ZSTD(1)), - execution_payload_parent_hash FixedString(66) CODEC(ZSTD(1)), - execution_payload_transactions_count Nullable(UInt32) CODEC(ZSTD(1)), - execution_payload_transactions_total_bytes Nullable(UInt32) CODEC(ZSTD(1)), - execution_payload_transactions_total_bytes_compressed Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon block from a beacon node.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN block_total_bytes 'The total bytes of the beacon block payload', -COMMENT COLUMN block_total_bytes_compressed 'The total bytes of the beacon block payload when compressed using snappy', -COMMENT COLUMN parent_root 'The root hash of the parent beacon block', -COMMENT COLUMN state_root 'The root hash of the beacon state at this block', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN eth1_data_block_hash 'The block hash of the associated execution block', -COMMENT COLUMN eth1_data_deposit_root 'The root of the deposit tree in the associated execution block', -COMMENT COLUMN execution_payload_block_hash 'The block hash of the execution payload', -COMMENT COLUMN execution_payload_block_number 'The block number of the execution payload', -COMMENT COLUMN execution_payload_fee_recipient 'The recipient of the fee for this execution payload', -COMMENT COLUMN execution_payload_state_root 'The state root of the execution payload', -COMMENT COLUMN execution_payload_parent_hash 'The parent hash of the execution payload', -COMMENT COLUMN execution_payload_transactions_count 'The transaction count of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes 'The transaction total bytes of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes_compressed 'The transaction total bytes of the execution payload when compressed using snappy', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block on cluster '{cluster}' AS canonical_beacon_block_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_local, rand()); - -CREATE TABLE default.canonical_beacon_block_proposer_slashing_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - signed_header_1_message_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_1_message_proposer_index UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_1_message_body_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_message_parent_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_message_state_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_signature String CODEC(ZSTD(1)), - signed_header_2_message_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_2_message_proposer_index UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_2_message_body_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_message_parent_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_message_state_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains proposer slashing from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN signed_header_1_message_slot 'The slot number from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_proposer_index 'The proposer index from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_body_root 'The body root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_parent_root 'The parent root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_state_root 'The state root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_signature 'The signature for the first signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_slot 'The slot number from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_proposer_index 'The proposer index from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_body_root 'The body root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_parent_root 'The parent root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_state_root 'The state root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_signature 'The signature for the second signed header in the slashing payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_proposer_slashing on cluster '{cluster}' AS canonical_beacon_block_proposer_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_proposer_slashing_local, rand()); - -CREATE TABLE default.canonical_beacon_block_attester_slashing_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - attestation_1_attesting_indices Array(UInt32) Codec(ZSTD(1)), - attestation_1_signature String CODEC(ZSTD(1)), - attestation_1_data_beacon_block_root FixedString(66) CODEC(ZSTD(1)), - attestation_1_data_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_index UInt32 Codec(ZSTD(1)), - attestation_1_data_source_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_source_root FixedString(66) CODEC(ZSTD(1)), - attestation_1_data_target_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_target_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_attesting_indices Array(UInt32) Codec(ZSTD(1)), - attestation_2_signature String CODEC(ZSTD(1)), - attestation_2_data_beacon_block_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_data_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_index UInt32 Codec(ZSTD(1)), - attestation_2_data_source_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_source_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_data_target_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_target_root FixedString(66) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains attester slashing from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN attestation_1_attesting_indices 'The attesting indices from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_signature 'The signature from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_beacon_block_root 'The beacon block root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_slot 'The slot number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_index 'The attestor index from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_source_epoch 'The source epoch number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_source_root 'The source root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_target_epoch 'The target epoch number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_target_root 'The target root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_2_attesting_indices 'The attesting indices from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_signature 'The signature from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_beacon_block_root 'The beacon block root from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_slot 'The slot number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_index 'The attestor index from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_source_epoch 'The source epoch number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_source_root 'The source root from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_target_epoch 'The target epoch number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_target_root 'The target root from the second attestation in the slashing payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_attester_slashing on cluster '{cluster}' AS canonical_beacon_block_attester_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_attester_slashing_local, rand()); - -CREATE TABLE default.canonical_beacon_block_bls_to_execution_change_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - exchanging_message_validator_index UInt32 CODEC(ZSTD(1)), - exchanging_message_from_bls_pubkey String CODEC(ZSTD(1)), - exchanging_message_to_execution_address FixedString(42) CODEC(ZSTD(1)), - exchanging_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains bls to execution change from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN exchanging_message_validator_index 'The validator index from the exchanging message', -COMMENT COLUMN exchanging_message_from_bls_pubkey 'The BLS public key from the exchanging message', -COMMENT COLUMN exchanging_message_to_execution_address 'The execution address from the exchanging message', -COMMENT COLUMN exchanging_signature 'The signature for the exchanging message', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' AS canonical_beacon_block_bls_to_execution_change_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_bls_to_execution_change_local, rand()); - -CREATE TABLE default.canonical_beacon_block_execution_transaction_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - position UInt32 CODEC(DoubleDelta, ZSTD(1)), - hash FixedString(66) Codec(ZSTD(1)), - from FixedString(42) Codec(ZSTD(1)), - to Nullable(FixedString(42)) Codec(ZSTD(1)), - nonce UInt64 Codec(ZSTD(1)), - gas_price UInt128 Codec(ZSTD(1)), - gas UInt64 Codec(ZSTD(1)), - value UInt128 Codec(ZSTD(1)), - type UInt8 Codec(ZSTD(1)), - size UInt32 Codec(ZSTD(1)), - call_data_size UInt32 Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution transaction from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN position 'The position of the transaction in the beacon block', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN type 'The type of the transaction', -COMMENT COLUMN size 'The size of the transaction data in bytes', -COMMENT COLUMN call_data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_execution_transaction on cluster '{cluster}' AS canonical_beacon_block_execution_transaction_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_execution_transaction_local, rand()); - -CREATE TABLE default.canonical_beacon_block_voluntary_exit_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - voluntary_exit_message_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - voluntary_exit_message_validator_index UInt32 CODEC(ZSTD(1)), - voluntary_exit_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a voluntary exit from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN voluntary_exit_message_epoch 'The epoch number from the exit message', -COMMENT COLUMN voluntary_exit_message_validator_index 'The validator index from the exit message', -COMMENT COLUMN voluntary_exit_signature 'The signature of the exit message', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_voluntary_exit on cluster '{cluster}' AS canonical_beacon_block_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_voluntary_exit_local, rand()); - -CREATE TABLE default.canonical_beacon_block_deposit_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - deposit_proof Array(String) CODEC(ZSTD(1)), - deposit_data_pubkey String CODEC(ZSTD(1)), - deposit_data_withdrawal_credentials FixedString(66) CODEC(ZSTD(1)), - deposit_data_amount UInt128 CODEC(ZSTD(1)), - deposit_data_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a deposit from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN deposit_proof 'The proof of the deposit data', -COMMENT COLUMN deposit_data_pubkey 'The BLS public key of the validator from the deposit data', -COMMENT COLUMN deposit_data_withdrawal_credentials 'The withdrawal credentials of the validator from the deposit data', -COMMENT COLUMN deposit_data_amount 'The amount of the deposit from the deposit data', -COMMENT COLUMN deposit_data_signature 'The signature of the deposit data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_deposit on cluster '{cluster}' AS canonical_beacon_block_deposit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_deposit_local, rand()); - -CREATE TABLE default.canonical_beacon_block_withdrawal_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - withdrawal_index UInt32 CODEC(ZSTD(1)), - withdrawal_validator_index UInt32 CODEC(ZSTD(1)), - withdrawal_address FixedString(42) Codec(ZSTD(1)), - withdrawal_amount UInt128 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a withdrawal from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN withdrawal_index 'The index of the withdrawal', -COMMENT COLUMN withdrawal_validator_index 'The validator index from the withdrawal data', -COMMENT COLUMN withdrawal_address 'The address of the account that is the withdrawal recipient', -COMMENT COLUMN withdrawal_amount 'The amount of the withdrawal from the withdrawal data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_withdrawal on cluster '{cluster}' AS canonical_beacon_block_withdrawal_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_withdrawal_local, rand()); diff --git a/deploy/migrations/clickhouse/015_cannon_replacing.down.sql b/deploy/migrations/clickhouse/015_cannon_replacing.down.sql deleted file mode 100644 index bc0b6f89..00000000 --- a/deploy/migrations/clickhouse/015_cannon_replacing.down.sql +++ /dev/null @@ -1,721 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_block on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_proposer_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_proposer_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_attester_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_attester_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_bls_to_execution_change_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_execution_transaction on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_execution_transaction_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_voluntary_exit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_voluntary_exit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_deposit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_deposit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_withdrawal on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_withdrawal_local on cluster '{cluster}' SYNC; - - -CREATE TABLE default.canonical_beacon_block_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - block_total_bytes Nullable(UInt32) CODEC(ZSTD(1)), - block_total_bytes_compressed Nullable(UInt32) CODEC(ZSTD(1)), - parent_root FixedString(66) CODEC(ZSTD(1)), - state_root FixedString(66) CODEC(ZSTD(1)), - proposer_index UInt32 CODEC(ZSTD(1)), - eth1_data_block_hash FixedString(66) CODEC(ZSTD(1)), - eth1_data_deposit_root FixedString(66) CODEC(ZSTD(1)), - execution_payload_block_hash FixedString(66) CODEC(ZSTD(1)), - execution_payload_block_number UInt32 CODEC(DoubleDelta, ZSTD(1)), - execution_payload_fee_recipient String CODEC(ZSTD(1)), - execution_payload_state_root FixedString(66) CODEC(ZSTD(1)), - execution_payload_parent_hash FixedString(66) CODEC(ZSTD(1)), - execution_payload_transactions_count Nullable(UInt32) CODEC(ZSTD(1)), - execution_payload_transactions_total_bytes Nullable(UInt32) CODEC(ZSTD(1)), - execution_payload_transactions_total_bytes_compressed Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon block from a beacon node.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN block_total_bytes 'The total bytes of the beacon block payload', -COMMENT COLUMN block_total_bytes_compressed 'The total bytes of the beacon block payload when compressed using snappy', -COMMENT COLUMN parent_root 'The root hash of the parent beacon block', -COMMENT COLUMN state_root 'The root hash of the beacon state at this block', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN eth1_data_block_hash 'The block hash of the associated execution block', -COMMENT COLUMN eth1_data_deposit_root 'The root of the deposit tree in the associated execution block', -COMMENT COLUMN execution_payload_block_hash 'The block hash of the execution payload', -COMMENT COLUMN execution_payload_block_number 'The block number of the execution payload', -COMMENT COLUMN execution_payload_fee_recipient 'The recipient of the fee for this execution payload', -COMMENT COLUMN execution_payload_state_root 'The state root of the execution payload', -COMMENT COLUMN execution_payload_parent_hash 'The parent hash of the execution payload', -COMMENT COLUMN execution_payload_transactions_count 'The transaction count of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes 'The transaction total bytes of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes_compressed 'The transaction total bytes of the execution payload when compressed using snappy', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block on cluster '{cluster}' AS canonical_beacon_block_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_local, rand()); - -CREATE TABLE default.canonical_beacon_block_proposer_slashing_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - signed_header_1_message_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_1_message_proposer_index UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_1_message_body_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_message_parent_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_message_state_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_signature String CODEC(ZSTD(1)), - signed_header_2_message_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_2_message_proposer_index UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_2_message_body_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_message_parent_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_message_state_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains proposer slashing from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN signed_header_1_message_slot 'The slot number from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_proposer_index 'The proposer index from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_body_root 'The body root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_parent_root 'The parent root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_state_root 'The state root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_signature 'The signature for the first signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_slot 'The slot number from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_proposer_index 'The proposer index from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_body_root 'The body root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_parent_root 'The parent root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_state_root 'The state root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_signature 'The signature for the second signed header in the slashing payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_proposer_slashing on cluster '{cluster}' AS canonical_beacon_block_proposer_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_proposer_slashing_local, rand()); - -CREATE TABLE default.canonical_beacon_block_attester_slashing_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - attestation_1_attesting_indices Array(UInt32) Codec(ZSTD(1)), - attestation_1_signature String CODEC(ZSTD(1)), - attestation_1_data_beacon_block_root FixedString(66) CODEC(ZSTD(1)), - attestation_1_data_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_index UInt32 Codec(ZSTD(1)), - attestation_1_data_source_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_source_root FixedString(66) CODEC(ZSTD(1)), - attestation_1_data_target_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_target_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_attesting_indices Array(UInt32) Codec(ZSTD(1)), - attestation_2_signature String CODEC(ZSTD(1)), - attestation_2_data_beacon_block_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_data_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_index UInt32 Codec(ZSTD(1)), - attestation_2_data_source_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_source_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_data_target_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_target_root FixedString(66) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains attester slashing from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN attestation_1_attesting_indices 'The attesting indices from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_signature 'The signature from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_beacon_block_root 'The beacon block root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_slot 'The slot number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_index 'The attestor index from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_source_epoch 'The source epoch number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_source_root 'The source root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_target_epoch 'The target epoch number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_target_root 'The target root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_2_attesting_indices 'The attesting indices from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_signature 'The signature from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_beacon_block_root 'The beacon block root from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_slot 'The slot number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_index 'The attestor index from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_source_epoch 'The source epoch number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_source_root 'The source root from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_target_epoch 'The target epoch number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_target_root 'The target root from the second attestation in the slashing payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_attester_slashing on cluster '{cluster}' AS canonical_beacon_block_attester_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_attester_slashing_local, rand()); - -CREATE TABLE default.canonical_beacon_block_bls_to_execution_change_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - exchanging_message_validator_index UInt32 CODEC(ZSTD(1)), - exchanging_message_from_bls_pubkey String CODEC(ZSTD(1)), - exchanging_message_to_execution_address FixedString(42) CODEC(ZSTD(1)), - exchanging_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains bls to execution change from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN exchanging_message_validator_index 'The validator index from the exchanging message', -COMMENT COLUMN exchanging_message_from_bls_pubkey 'The BLS public key from the exchanging message', -COMMENT COLUMN exchanging_message_to_execution_address 'The execution address from the exchanging message', -COMMENT COLUMN exchanging_signature 'The signature for the exchanging message', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' AS canonical_beacon_block_bls_to_execution_change_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_bls_to_execution_change_local, rand()); - -CREATE TABLE default.canonical_beacon_block_execution_transaction_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - position UInt32 CODEC(DoubleDelta, ZSTD(1)), - hash FixedString(66) Codec(ZSTD(1)), - from FixedString(42) Codec(ZSTD(1)), - to Nullable(FixedString(42)) Codec(ZSTD(1)), - nonce UInt64 Codec(ZSTD(1)), - gas_price UInt128 Codec(ZSTD(1)), - gas UInt64 Codec(ZSTD(1)), - value UInt128 Codec(ZSTD(1)), - type UInt8 Codec(ZSTD(1)), - size UInt32 Codec(ZSTD(1)), - call_data_size UInt32 Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution transaction from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN position 'The position of the transaction in the beacon block', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN type 'The type of the transaction', -COMMENT COLUMN size 'The size of the transaction data in bytes', -COMMENT COLUMN call_data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_execution_transaction on cluster '{cluster}' AS canonical_beacon_block_execution_transaction_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_execution_transaction_local, rand()); - -CREATE TABLE default.canonical_beacon_block_voluntary_exit_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - voluntary_exit_message_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - voluntary_exit_message_validator_index UInt32 CODEC(ZSTD(1)), - voluntary_exit_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a voluntary exit from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN voluntary_exit_message_epoch 'The epoch number from the exit message', -COMMENT COLUMN voluntary_exit_message_validator_index 'The validator index from the exit message', -COMMENT COLUMN voluntary_exit_signature 'The signature of the exit message', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_voluntary_exit on cluster '{cluster}' AS canonical_beacon_block_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_voluntary_exit_local, rand()); - -CREATE TABLE default.canonical_beacon_block_deposit_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - deposit_proof Array(String) CODEC(ZSTD(1)), - deposit_data_pubkey String CODEC(ZSTD(1)), - deposit_data_withdrawal_credentials FixedString(66) CODEC(ZSTD(1)), - deposit_data_amount UInt128 CODEC(ZSTD(1)), - deposit_data_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a deposit from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN deposit_proof 'The proof of the deposit data', -COMMENT COLUMN deposit_data_pubkey 'The BLS public key of the validator from the deposit data', -COMMENT COLUMN deposit_data_withdrawal_credentials 'The withdrawal credentials of the validator from the deposit data', -COMMENT COLUMN deposit_data_amount 'The amount of the deposit from the deposit data', -COMMENT COLUMN deposit_data_signature 'The signature of the deposit data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_deposit on cluster '{cluster}' AS canonical_beacon_block_deposit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_deposit_local, rand()); - -CREATE TABLE default.canonical_beacon_block_withdrawal_local on cluster '{cluster}' -( - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - withdrawal_index UInt32 CODEC(ZSTD(1)), - withdrawal_validator_index UInt32 CODEC(ZSTD(1)), - withdrawal_address FixedString(42) Codec(ZSTD(1)), - withdrawal_amount UInt128 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a withdrawal from a beacon block.', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN withdrawal_index 'The index of the withdrawal', -COMMENT COLUMN withdrawal_validator_index 'The validator index from the withdrawal data', -COMMENT COLUMN withdrawal_address 'The address of the account that is the withdrawal recipient', -COMMENT COLUMN withdrawal_amount 'The amount of the withdrawal from the withdrawal data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_withdrawal on cluster '{cluster}' AS canonical_beacon_block_withdrawal_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_withdrawal_local, rand()); diff --git a/deploy/migrations/clickhouse/015_cannon_replacing.up.sql b/deploy/migrations/clickhouse/015_cannon_replacing.up.sql deleted file mode 100644 index 1b3207ad..00000000 --- a/deploy/migrations/clickhouse/015_cannon_replacing.up.sql +++ /dev/null @@ -1,750 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_block on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_proposer_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_proposer_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_attester_slashing on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_attester_slashing_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_bls_to_execution_change_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_execution_transaction on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_execution_transaction_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_voluntary_exit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_voluntary_exit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_deposit on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_deposit_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_withdrawal on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_block_withdrawal_local on cluster '{cluster}' SYNC; - -CREATE TABLE default.canonical_beacon_block_local on cluster '{cluster}' -( - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - block_total_bytes Nullable(UInt32) CODEC(ZSTD(1)), - block_total_bytes_compressed Nullable(UInt32) CODEC(ZSTD(1)), - parent_root FixedString(66) CODEC(ZSTD(1)), - state_root FixedString(66) CODEC(ZSTD(1)), - proposer_index UInt32 CODEC(ZSTD(1)), - eth1_data_block_hash FixedString(66) CODEC(ZSTD(1)), - eth1_data_deposit_root FixedString(66) CODEC(ZSTD(1)), - execution_payload_block_hash FixedString(66) CODEC(ZSTD(1)), - execution_payload_block_number UInt32 CODEC(DoubleDelta, ZSTD(1)), - execution_payload_fee_recipient String CODEC(ZSTD(1)), - execution_payload_state_root FixedString(66) CODEC(ZSTD(1)), - execution_payload_parent_hash FixedString(66) CODEC(ZSTD(1)), - execution_payload_transactions_count Nullable(UInt32) CODEC(ZSTD(1)), - execution_payload_transactions_total_bytes Nullable(UInt32) CODEC(ZSTD(1)), - execution_payload_transactions_total_bytes_compressed Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name); - -ALTER TABLE default.canonical_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon block from a beacon node.', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN block_total_bytes 'The total bytes of the beacon block payload', -COMMENT COLUMN block_total_bytes_compressed 'The total bytes of the beacon block payload when compressed using snappy', -COMMENT COLUMN parent_root 'The root hash of the parent beacon block', -COMMENT COLUMN state_root 'The root hash of the beacon state at this block', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN eth1_data_block_hash 'The block hash of the associated execution block', -COMMENT COLUMN eth1_data_deposit_root 'The root of the deposit tree in the associated execution block', -COMMENT COLUMN execution_payload_block_hash 'The block hash of the execution payload', -COMMENT COLUMN execution_payload_block_number 'The block number of the execution payload', -COMMENT COLUMN execution_payload_fee_recipient 'The recipient of the fee for this execution payload', -COMMENT COLUMN execution_payload_state_root 'The state root of the execution payload', -COMMENT COLUMN execution_payload_parent_hash 'The parent hash of the execution payload', -COMMENT COLUMN execution_payload_transactions_count 'The transaction count of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes 'The transaction total bytes of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes_compressed 'The transaction total bytes of the execution payload when compressed using snappy', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block on cluster '{cluster}' AS canonical_beacon_block_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_local, rand()); - -CREATE TABLE default.canonical_beacon_block_proposer_slashing_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - signed_header_1_message_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_1_message_proposer_index UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_1_message_body_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_message_parent_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_message_state_root FixedString(66) CODEC(ZSTD(1)), - signed_header_1_signature String CODEC(ZSTD(1)), - signed_header_2_message_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_2_message_proposer_index UInt32 CODEC(DoubleDelta, ZSTD(1)), - signed_header_2_message_body_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_message_parent_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_message_state_root FixedString(66) CODEC(ZSTD(1)), - signed_header_2_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains proposer slashing from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN signed_header_1_message_slot 'The slot number from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_proposer_index 'The proposer index from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_body_root 'The body root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_parent_root 'The parent root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_message_state_root 'The state root from the first signed header in the slashing payload', -COMMENT COLUMN signed_header_1_signature 'The signature for the first signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_slot 'The slot number from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_proposer_index 'The proposer index from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_body_root 'The body root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_parent_root 'The parent root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_message_state_root 'The state root from the second signed header in the slashing payload', -COMMENT COLUMN signed_header_2_signature 'The signature for the second signed header in the slashing payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_proposer_slashing on cluster '{cluster}' AS canonical_beacon_block_proposer_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_proposer_slashing_local, rand()); - -CREATE TABLE default.canonical_beacon_block_attester_slashing_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - attestation_1_attesting_indices Array(UInt32) Codec(ZSTD(1)), - attestation_1_signature String CODEC(ZSTD(1)), - attestation_1_data_beacon_block_root FixedString(66) CODEC(ZSTD(1)), - attestation_1_data_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_index UInt32 Codec(ZSTD(1)), - attestation_1_data_source_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_source_root FixedString(66) CODEC(ZSTD(1)), - attestation_1_data_target_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_1_data_target_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_attesting_indices Array(UInt32) Codec(ZSTD(1)), - attestation_2_signature String CODEC(ZSTD(1)), - attestation_2_data_beacon_block_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_data_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_index UInt32 Codec(ZSTD(1)), - attestation_2_data_source_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_source_root FixedString(66) CODEC(ZSTD(1)), - attestation_2_data_target_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - attestation_2_data_target_root FixedString(66) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains attester slashing from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN attestation_1_attesting_indices 'The attesting indices from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_signature 'The signature from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_beacon_block_root 'The beacon block root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_slot 'The slot number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_index 'The attestor index from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_source_epoch 'The source epoch number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_source_root 'The source root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_target_epoch 'The target epoch number from the first attestation in the slashing payload', -COMMENT COLUMN attestation_1_data_target_root 'The target root from the first attestation in the slashing payload', -COMMENT COLUMN attestation_2_attesting_indices 'The attesting indices from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_signature 'The signature from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_beacon_block_root 'The beacon block root from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_slot 'The slot number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_index 'The attestor index from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_source_epoch 'The source epoch number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_source_root 'The source root from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_target_epoch 'The target epoch number from the second attestation in the slashing payload', -COMMENT COLUMN attestation_2_data_target_root 'The target root from the second attestation in the slashing payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_attester_slashing on cluster '{cluster}' AS canonical_beacon_block_attester_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_attester_slashing_local, rand()); - -CREATE TABLE default.canonical_beacon_block_bls_to_execution_change_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - exchanging_message_validator_index UInt32 CODEC(ZSTD(1)), - exchanging_message_from_bls_pubkey String CODEC(ZSTD(1)), - exchanging_message_to_execution_address FixedString(42) CODEC(ZSTD(1)), - exchanging_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains bls to execution change from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN exchanging_message_validator_index 'The validator index from the exchanging message', -COMMENT COLUMN exchanging_message_from_bls_pubkey 'The BLS public key from the exchanging message', -COMMENT COLUMN exchanging_message_to_execution_address 'The execution address from the exchanging message', -COMMENT COLUMN exchanging_signature 'The signature for the exchanging message', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' AS canonical_beacon_block_bls_to_execution_change_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_bls_to_execution_change_local, rand()); - -CREATE TABLE default.canonical_beacon_block_execution_transaction_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - position UInt32 CODEC(DoubleDelta, ZSTD(1)), - hash FixedString(66) Codec(ZSTD(1)), - from FixedString(42) Codec(ZSTD(1)), - to Nullable(FixedString(42)) Codec(ZSTD(1)), - nonce UInt64 Codec(ZSTD(1)), - gas_price UInt128 Codec(ZSTD(1)), - gas UInt64 Codec(ZSTD(1)), - value UInt128 Codec(ZSTD(1)), - type UInt8 Codec(ZSTD(1)), - size UInt32 Codec(ZSTD(1)), - call_data_size UInt32 Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution transaction from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN position 'The position of the transaction in the beacon block', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN type 'The type of the transaction', -COMMENT COLUMN size 'The size of the transaction data in bytes', -COMMENT COLUMN call_data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_execution_transaction on cluster '{cluster}' AS canonical_beacon_block_execution_transaction_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_execution_transaction_local, rand()); - -CREATE TABLE default.canonical_beacon_block_voluntary_exit_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - voluntary_exit_message_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - voluntary_exit_message_validator_index UInt32 CODEC(ZSTD(1)), - voluntary_exit_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a voluntary exit from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN voluntary_exit_message_epoch 'The epoch number from the exit message', -COMMENT COLUMN voluntary_exit_message_validator_index 'The validator index from the exit message', -COMMENT COLUMN voluntary_exit_signature 'The signature of the exit message', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_voluntary_exit on cluster '{cluster}' AS canonical_beacon_block_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_voluntary_exit_local, rand()); - -CREATE TABLE default.canonical_beacon_block_deposit_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - deposit_proof Array(String) CODEC(ZSTD(1)), - deposit_data_pubkey String CODEC(ZSTD(1)), - deposit_data_withdrawal_credentials FixedString(66) CODEC(ZSTD(1)), - deposit_data_amount UInt128 CODEC(ZSTD(1)), - deposit_data_signature String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a deposit from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN deposit_proof 'The proof of the deposit data', -COMMENT COLUMN deposit_data_pubkey 'The BLS public key of the validator from the deposit data', -COMMENT COLUMN deposit_data_withdrawal_credentials 'The withdrawal credentials of the validator from the deposit data', -COMMENT COLUMN deposit_data_amount 'The amount of the deposit from the deposit data', -COMMENT COLUMN deposit_data_signature 'The signature of the deposit data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_deposit on cluster '{cluster}' AS canonical_beacon_block_deposit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_deposit_local, rand()); - -CREATE TABLE default.canonical_beacon_block_withdrawal_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_version LowCardinality(String), - withdrawal_index UInt32 CODEC(ZSTD(1)), - withdrawal_validator_index UInt32 CODEC(ZSTD(1)), - withdrawal_address FixedString(42) Codec(ZSTD(1)), - withdrawal_amount UInt128 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a withdrawal from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN withdrawal_index 'The index of the withdrawal', -COMMENT COLUMN withdrawal_validator_index 'The validator index from the withdrawal data', -COMMENT COLUMN withdrawal_address 'The address of the account that is the withdrawal recipient', -COMMENT COLUMN withdrawal_amount 'The amount of the withdrawal from the withdrawal data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_block_withdrawal on cluster '{cluster}' AS canonical_beacon_block_withdrawal_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_withdrawal_local, rand()); diff --git a/deploy/migrations/clickhouse/016_blob_sidecar.down.sql b/deploy/migrations/clickhouse/016_blob_sidecar.down.sql deleted file mode 100644 index 3550bb15..00000000 --- a/deploy/migrations/clickhouse/016_blob_sidecar.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_events_blob_sidecar on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_blob_sidecar_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/016_blob_sidecar.up.sql b/deploy/migrations/clickhouse/016_blob_sidecar.up.sql deleted file mode 100644 index 8b6b34ee..00000000 --- a/deploy/migrations/clickhouse/016_blob_sidecar.up.sql +++ /dev/null @@ -1,74 +0,0 @@ -CREATE TABLE beacon_api_eth_v1_events_blob_sidecar_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - block_root FixedString(66) Codec(ZSTD(1)), - blob_index UInt64 Codec(ZSTD(1)), - kzg_commitment FixedString(98) Codec(ZSTD(1)), - versioned_hash FixedString(66) Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name); - -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN blob_index 'The index of blob sidecar in the beacon API event stream payload', -COMMENT COLUMN kzg_commitment 'The KZG commitment in the beacon API event stream payload', -COMMENT COLUMN versioned_hash 'The versioned hash in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v1_events_blob_sidecar on cluster '{cluster}' AS beacon_api_eth_v1_events_blob_sidecar_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_blob_sidecar_local, rand()); diff --git a/deploy/migrations/clickhouse/017_cannon_blockprint.down.sql b/deploy/migrations/clickhouse/017_cannon_blockprint.down.sql deleted file mode 100644 index d06ac31a..00000000 --- a/deploy/migrations/clickhouse/017_cannon_blockprint.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_block_classification on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_block_classification_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/017_cannon_blockprint.up.sql b/deploy/migrations/clickhouse/017_cannon_blockprint.up.sql deleted file mode 100644 index 0ca96935..00000000 --- a/deploy/migrations/clickhouse/017_cannon_blockprint.up.sql +++ /dev/null @@ -1,87 +0,0 @@ -CREATE TABLE default.beacon_block_classification_local on cluster '{cluster}' -( - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - best_guess_single LowCardinality(String), - best_guess_multi LowCardinality(String), - client_probability_uncertain Float32 CODEC(ZSTD(1)), - client_probability_prysm Float32 CODEC(ZSTD(1)), - client_probability_teku Float32 CODEC(ZSTD(1)), - client_probability_nimbus Float32 CODEC(ZSTD(1)), - client_probability_lodestar Float32 CODEC(ZSTD(1)), - client_probability_grandine Float32 CODEC(ZSTD(1)), - client_probability_lighthouse Float32 CODEC(ZSTD(1)), - proposer_index UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name); - -ALTER TABLE default.beacon_block_classification_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon block classification for a given slot. This is a best guess based on the client probabilities of the proposer. This is not guaranteed to be correct.', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block classification', -COMMENT COLUMN slot 'The slot number from beacon block classification', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block classification', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN best_guess_single 'The best guess of the client that generated the beacon block', -COMMENT COLUMN best_guess_multi 'The best guess of the clients that generated the beacon block. This value will typically equal the best_guess_single value, but when multiple clients have high probabilities, this value will have multiple eg. "prysm or lighthouse"', -COMMENT COLUMN client_probability_uncertain 'The probability that the client that generated the beacon block is uncertain', -COMMENT COLUMN client_probability_prysm 'The probability that the client that generated the beacon block is Prysm', -COMMENT COLUMN client_probability_teku 'The probability that the client that generated the beacon block is Teku', -COMMENT COLUMN client_probability_nimbus 'The probability that the client that generated the beacon block is Nimbus', -COMMENT COLUMN client_probability_lodestar 'The probability that the client that generated the beacon block is Lodestar', -COMMENT COLUMN client_probability_grandine 'The probability that the client that generated the beacon block is Grandine', -COMMENT COLUMN client_probability_lighthouse 'The probability that the client that generated the beacon block is Lighthouse', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_block_classification on cluster '{cluster}' AS beacon_block_classification_local -ENGINE = Distributed('{cluster}', default, beacon_block_classification_local, rand()); diff --git a/deploy/migrations/clickhouse/018_cannon_blob_sidecar.down.sql b/deploy/migrations/clickhouse/018_cannon_blob_sidecar.down.sql deleted file mode 100644 index aed2ce01..00000000 --- a/deploy/migrations/clickhouse/018_cannon_blob_sidecar.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_blob_sidecar on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_blob_sidecar_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/018_cannon_blob_sidecar.up.sql b/deploy/migrations/clickhouse/018_cannon_blob_sidecar.up.sql deleted file mode 100644 index ac468079..00000000 --- a/deploy/migrations/clickhouse/018_cannon_blob_sidecar.up.sql +++ /dev/null @@ -1,83 +0,0 @@ -CREATE TABLE default.canonical_beacon_blob_sidecar_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - block_parent_root FixedString(66) CODEC(ZSTD(1)), - kzg_commitment FixedString(98) Codec(ZSTD(1)), - kzg_proof FixedString(98) Codec(ZSTD(1)), - proposer_index UInt32 Codec(ZSTD(1)), - blob_index UInt64 Codec(ZSTD(1)), - blob_size UInt32 Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a blob sidecar from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_parent_root 'The root hash of the parent beacon block', -COMMENT COLUMN kzg_commitment 'The KZG commitment in the blob sidecar payload', -COMMENT COLUMN kzg_proof 'The KZG proof in the blob sidecar payload', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN blob_index 'The index of blob sidecar in the blob sidecar payload', -COMMENT COLUMN blob_size 'The total bytes of the blob', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_blob_sidecar on cluster '{cluster}' AS canonical_beacon_blob_sidecar_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_blob_sidecar_local, rand()); diff --git a/deploy/migrations/clickhouse/019_blob_sidecar_versioned_hash.down.sql b/deploy/migrations/clickhouse/019_blob_sidecar_versioned_hash.down.sql deleted file mode 100644 index 65e06a12..00000000 --- a/deploy/migrations/clickhouse/019_blob_sidecar_versioned_hash.down.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE canonical_beacon_blob_sidecar_local on cluster '{cluster}' - DROP COLUMN versioned_hash; - -ALTER TABLE canonical_beacon_blob_sidecar on cluster '{cluster}' - DROP COLUMN versioned_hash; diff --git a/deploy/migrations/clickhouse/019_blob_sidecar_versioned_hash.up.sql b/deploy/migrations/clickhouse/019_blob_sidecar_versioned_hash.up.sql deleted file mode 100644 index 8260370a..00000000 --- a/deploy/migrations/clickhouse/019_blob_sidecar_versioned_hash.up.sql +++ /dev/null @@ -1,11 +0,0 @@ -ALTER TABLE canonical_beacon_blob_sidecar_local on cluster '{cluster}' -ADD COLUMN versioned_hash FixedString(66) Codec(ZSTD(1)) AFTER block_parent_root; - -ALTER TABLE canonical_beacon_blob_sidecar on cluster '{cluster}' -ADD COLUMN versioned_hash FixedString(66) Codec(ZSTD(1)) AFTER block_parent_root; - -ALTER TABLE canonical_beacon_blob_sidecar_local on cluster '{cluster}' -COMMENT COLUMN versioned_hash 'The versioned hash in the beacon API event stream payload'; - -ALTER TABLE canonical_beacon_blob_sidecar on cluster '{cluster}' -COMMENT COLUMN versioned_hash 'The versioned hash in the beacon API event stream payload'; diff --git a/deploy/migrations/clickhouse/020_imports.down.sql b/deploy/migrations/clickhouse/020_imports.down.sql deleted file mode 100644 index 8b2e92f0..00000000 --- a/deploy/migrations/clickhouse/020_imports.down.sql +++ /dev/null @@ -1,6 +0,0 @@ -DROP TABLE IF EXISTS imported_sources on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS imported_sources_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS mempool_dumpster_transaction on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS mempool_dumpster_transaction_local on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS block_native_mempool_transaction on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS block_native_mempool_transaction_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/020_imports.up.sql b/deploy/migrations/clickhouse/020_imports.up.sql deleted file mode 100644 index 7066d03e..00000000 --- a/deploy/migrations/clickhouse/020_imports.up.sql +++ /dev/null @@ -1,137 +0,0 @@ -CREATE TABLE default.imported_sources_local on cluster '{cluster}' -( - create_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - target_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source LowCardinality(String) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(create_date_time) -ORDER BY (create_date_time, source); - -ALTER TABLE default.imported_sources_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'This table contains the list of sources that have been imported into the database', -COMMENT COLUMN create_date_time 'Creation date of this row', -COMMENT COLUMN target_date_time 'The date of the data that was imported', -COMMENT COLUMN source 'Source of the data that was imported'; - -CREATE TABLE imported_sources on cluster '{cluster}' AS imported_sources_local -ENGINE = Distributed('{cluster}', default, imported_sources_local, rand()); - -CREATE TABLE default.mempool_dumpster_transaction_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - timestamp DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - hash FixedString(66) Codec(ZSTD(1)), - chain_id UInt32 Codec(ZSTD(1)), - from FixedString(42) Codec(ZSTD(1)), - to Nullable(FixedString(42)) Codec(ZSTD(1)), - value UInt128 Codec(ZSTD(1)), - nonce UInt64 Codec(ZSTD(1)), - gas UInt64 Codec(ZSTD(1)), - gas_price UInt128 Codec(ZSTD(1)), - gas_tip_cap Nullable(UInt128) Codec(ZSTD(1)), - gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)), - data_size UInt32 Codec(ZSTD(1)), - data_4bytes Nullable(FixedString(10)) Codec(ZSTD(1)), - sources Array(LowCardinality(String)), - included_at_block_height Nullable(UInt64) Codec(ZSTD(1)), - included_block_timestamp Nullable(DateTime64(3)) Codec(DoubleDelta, ZSTD(1)), - inclusion_delay_ms Nullable(Int64) Codec(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(timestamp) -ORDER BY (timestamp, unique_key, chain_id); - -ALTER TABLE default.mempool_dumpster_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions from mempool dumpster dataset. Following the parquet schema with some additions', -COMMENT COLUMN unique_key 'Unique key for the row, this is outside the source data and used for deduplication', -COMMENT COLUMN updated_date_time 'When this row was last updated, this is outside the source data and used for deduplication', -COMMENT COLUMN timestamp 'Timestamp of the transaction', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN chain_id 'The chain id of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas_tip_cap 'The gas tip cap of the transaction in wei', -COMMENT COLUMN gas_fee_cap 'The gas fee cap of the transaction in wei', -COMMENT COLUMN data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN data_4bytes 'The first 4 bytes of the call data of the transaction', -COMMENT COLUMN sources 'The sources that saw this transaction in their mempool', -COMMENT COLUMN included_at_block_height 'The block height at which this transaction was included', -COMMENT COLUMN included_block_timestamp 'The timestamp of the block at which this transaction was included', -COMMENT COLUMN inclusion_delay_ms 'The delay between the transaction timestamp and the block timestamp'; - -CREATE TABLE mempool_dumpster_transaction on cluster '{cluster}' AS mempool_dumpster_transaction_local -ENGINE = Distributed('{cluster}', default, mempool_dumpster_transaction_local, rand()); - -CREATE TABLE default.block_native_mempool_transaction_local on cluster '{cluster}' -( - unique_key Int64 CODEC(ZSTD(1)), - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - detecttime DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - hash FixedString(66) Codec(ZSTD(1)), - status LowCardinality(String), - region LowCardinality(String), - reorg Nullable(FixedString(66)) Codec(ZSTD(1)), - replace Nullable(FixedString(66)) Codec(ZSTD(1)), - curblocknumber Nullable(UInt64) Codec(ZSTD(1)), - failurereason Nullable(String) Codec(ZSTD(1)), - blockspending Nullable(UInt64) Codec(ZSTD(1)), - timepending Nullable(UInt64) Codec(ZSTD(1)), - nonce UInt64 Codec(ZSTD(1)), - gas UInt64 Codec(ZSTD(1)), - gasprice UInt128 Codec(ZSTD(1)), - value UInt128 Codec(ZSTD(1)), - toaddress Nullable(FixedString(42)) Codec(ZSTD(1)), - fromaddress FixedString(42) Codec(ZSTD(1)), - datasize UInt32 Codec(ZSTD(1)), - data4bytes Nullable(FixedString(10)) Codec(ZSTD(1)), - network LowCardinality(String), - type UInt8 Codec(ZSTD(1)), - maxpriorityfeepergas Nullable(UInt128) Codec(ZSTD(1)), - maxfeepergas Nullable(UInt128) Codec(ZSTD(1)), - basefeepergas Nullable(UInt128) Codec(ZSTD(1)), - dropreason Nullable(String) Codec(ZSTD(1)), - rejectionreason Nullable(String) Codec(ZSTD(1)), - stuck Bool Codec(ZSTD(1)), - gasused Nullable(UInt64) Codec(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(detecttime) -ORDER BY (detecttime, unique_key, network); - -ALTER TABLE default.block_native_mempool_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions from block native mempool dataset', -COMMENT COLUMN unique_key 'Unique key for the row, this is outside the source data and used for deduplication', -COMMENT COLUMN updated_date_time 'When this row was last updated, this is outside the source data and used for deduplication', -COMMENT COLUMN detecttime 'Timestamp that the transaction was detected in mempool', -COMMENT COLUMN hash 'Unique identifier hash for a given transaction', -COMMENT COLUMN status 'Status of the transaction', -COMMENT COLUMN region 'The geographic region for the node that detected the transaction', -COMMENT COLUMN reorg 'If there was a reorg, refers to the blockhash of the reorg', -COMMENT COLUMN replace 'If the transaction was replaced (speedup/cancel), the transaction hash of the replacement', -COMMENT COLUMN curblocknumber 'The block number the event was detected in', -COMMENT COLUMN failurereason 'If a transaction failed, this field provides contextual information', -COMMENT COLUMN blockspending 'If a transaction was finalized (confirmed, failed), this refers to the number of blocks that the transaction was waiting to get on-chain', -COMMENT COLUMN timepending 'If a transaction was finalized (confirmed, failed), this refers to the time in milliseconds that the transaction was waiting to get on-chain', -COMMENT COLUMN nonce 'A unique number which counts the number of transactions sent from a given address', -COMMENT COLUMN gas 'The maximum number of gas units allowed for the transaction', -COMMENT COLUMN gasprice 'The price offered to the miner/validator per unit of gas. Denominated in wei', -COMMENT COLUMN value 'The amount of ETH transferred or sent to contract. Denominated in wei', -COMMENT COLUMN toaddress 'The destination of a given transaction', -COMMENT COLUMN fromaddress 'The source/initiator of a given transaction', -COMMENT COLUMN datasize 'The size of the call data of the transaction in bytes', -COMMENT COLUMN data4bytes 'The first 4 bytes of the call data of the transaction', -COMMENT COLUMN network 'The specific Ethereum network used', -COMMENT COLUMN type 'Post EIP-1559, this indicates how the gas parameters are submitted to the network: - type 0 - legacy - type 1 - usage of access lists according to EIP-2930 - type 2 - using maxpriorityfeepergas and maxfeepergas', -COMMENT COLUMN maxpriorityfeepergas 'The maximum value for a tip offered to the miner/validator per unit of gas. The actual tip paid can be lower if (maxfee - basefee) < maxpriorityfee. Denominated in wei', -COMMENT COLUMN maxfeepergas 'The maximum value for the transaction fee (including basefee and tip) offered to the miner/validator per unit of gas. Denominated in wei', -COMMENT COLUMN basefeepergas 'The fee per unit of gas paid and burned for the curblocknumber. This fee is algorithmically determined. Denominated in wei', -COMMENT COLUMN dropreason 'If the transaction was dropped from the mempool, this describes the contextual reason for the drop', -COMMENT COLUMN rejectionreason 'If the transaction was rejected from the mempool, this describes the contextual reason for the rejection', -COMMENT COLUMN stuck 'A transaction was detected in the queued area of the mempool and is not eligible for inclusion in a block', -COMMENT COLUMN gasused 'If the transaction was published on-chain, this value indicates the amount of gas that was actually consumed. Denominated in wei'; - -CREATE TABLE block_native_mempool_transaction on cluster '{cluster}' AS block_native_mempool_transaction_local -ENGINE = Distributed('{cluster}', default, block_native_mempool_transaction_local, rand()); diff --git a/deploy/migrations/clickhouse/021_fix_beacon_block.down.sql b/deploy/migrations/clickhouse/021_fix_beacon_block.down.sql deleted file mode 100644 index 83802d3e..00000000 --- a/deploy/migrations/clickhouse/021_fix_beacon_block.down.sql +++ /dev/null @@ -1,7 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' - DROP COLUMN meta_execution_fork_id_hash, - DROP COLUMN meta_execution_fork_id_next; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' - DROP COLUMN meta_execution_fork_id_hash, - DROP COLUMN meta_execution_fork_id_next; diff --git a/deploy/migrations/clickhouse/021_fix_beacon_block.up.sql b/deploy/migrations/clickhouse/021_fix_beacon_block.up.sql deleted file mode 100644 index 7850731c..00000000 --- a/deploy/migrations/clickhouse/021_fix_beacon_block.up.sql +++ /dev/null @@ -1,7 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS meta_execution_fork_id_hash LowCardinality(String) AFTER meta_network_name, - ADD COLUMN IF NOT EXISTS meta_execution_fork_id_next LowCardinality(String) AFTER meta_execution_fork_id_hash; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS meta_execution_fork_id_hash LowCardinality(String) AFTER meta_network_name, - ADD COLUMN IF NOT EXISTS meta_execution_fork_id_next LowCardinality(String) AFTER meta_execution_fork_id_hash; diff --git a/deploy/migrations/clickhouse/022_beacon_p2p_attestation.down.sql b/deploy/migrations/clickhouse/022_beacon_p2p_attestation.down.sql deleted file mode 100644 index 1ced0e12..00000000 --- a/deploy/migrations/clickhouse/022_beacon_p2p_attestation.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_p2p_attestation on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_p2p_attestation_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/022_beacon_p2p_attestation.up.sql b/deploy/migrations/clickhouse/022_beacon_p2p_attestation.up.sql deleted file mode 100644 index a2e91700..00000000 --- a/deploy/migrations/clickhouse/022_beacon_p2p_attestation.up.sql +++ /dev/null @@ -1,131 +0,0 @@ -CREATE TABLE default.beacon_p2p_attestation_local on cluster '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - committee_index LowCardinality(String), - attesting_validator_index Nullable(UInt32) Codec(ZSTD(1)), - attesting_validator_committee_index LowCardinality(String), - signature String Codec(ZSTD(1)), - aggregation_bits String Codec(ZSTD(1)), - beacon_block_root FixedString(66) Codec(ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - source_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source_root FixedString(66) Codec(ZSTD(1)), - target_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - target_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - target_root FixedString(66) Codec(ZSTD(1)), - attestation_subnet LowCardinality(String), - validated Bool, - peer_id String CODEC(ZSTD(1)), - peer_latency UInt32 Codec(ZSTD(1)), - peer_version LowCardinality(String), - peer_version_major LowCardinality(String), - peer_version_minor LowCardinality(String), - peer_version_patch LowCardinality(String), - peer_implementation LowCardinality(String), - peer_ip Nullable(IPv6) CODEC(ZSTD(1)), - peer_geo_city LowCardinality(String) CODEC(ZSTD(1)), - peer_geo_country LowCardinality(String) CODEC(ZSTD(1)), - peer_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - peer_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - peer_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - peer_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - peer_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - peer_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.beacon_p2p_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon chain P2P "attestation" data', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon P2P payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN committee_index 'The committee index in the beacon P2P payload', -COMMENT COLUMN attesting_validator_index 'The index of the validator attesting to the event', -COMMENT COLUMN attesting_validator_committee_index 'The committee index of the attesting validator', -COMMENT COLUMN signature 'The signature of the event in the beacon P2P payload', -COMMENT COLUMN aggregation_bits 'The aggregation bits of the event in the beacon P2P payload', -COMMENT COLUMN beacon_block_root 'The beacon block root hash in the beacon P2P payload', -COMMENT COLUMN epoch 'The epoch number in the beacon P2P payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN source_epoch 'The source epoch number in the beacon P2P payload', -COMMENT COLUMN source_epoch_start_date_time 'The wall clock time when the source epoch started', -COMMENT COLUMN source_root 'The source beacon block root hash in the beacon P2P payload', -COMMENT COLUMN target_epoch 'The target epoch number in the beacon P2P payload', -COMMENT COLUMN target_epoch_start_date_time 'The wall clock time when the target epoch started', -COMMENT COLUMN target_root 'The target beacon block root hash in the beacon P2P payload', -COMMENT COLUMN attestation_subnet 'The attestation subnet the attestation was gossiped on', -COMMENT COLUMN validated 'Whether the attestation was validated by the client', -COMMENT COLUMN peer_id 'The originating peer ID for the gossiped data', -COMMENT COLUMN peer_latency 'The latency of the peer that gossiped the data', -COMMENT COLUMN peer_version 'Peer client version that gossiped the data', -COMMENT COLUMN peer_version_major 'Peer client major version that gossiped the data', -COMMENT COLUMN peer_version_minor 'Peer client minor version that gossiped the data', -COMMENT COLUMN peer_version_patch 'Peer client patch version that gossiped the data', -COMMENT COLUMN peer_implementation 'Peer client implementation that gossiped the data', -COMMENT COLUMN peer_ip 'IP address of the peer that gossiped the data', -COMMENT COLUMN peer_geo_city 'City of the peer that gossiped the data', -COMMENT COLUMN peer_geo_country 'Country of the peer that gossiped the data', -COMMENT COLUMN peer_geo_country_code 'Country code of the peer that gossiped the data', -COMMENT COLUMN peer_geo_continent_code 'Continent code of the peer that gossiped the data', -COMMENT COLUMN peer_geo_longitude 'Longitude of the peer that gossiped the data', -COMMENT COLUMN peer_geo_latitude 'Latitude of the peer that gossiped the data', -COMMENT COLUMN peer_geo_autonomous_system_number 'Autonomous system number of the peer that gossiped the data', -COMMENT COLUMN peer_geo_autonomous_system_organization 'Autonomous system organization of the peer that gossiped the data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_p2p_attestation on cluster '{cluster}' AS beacon_p2p_attestation_local -ENGINE = Distributed('{cluster}', default, beacon_p2p_attestation_local, rand()); diff --git a/deploy/migrations/clickhouse/023_cannon_proposer_duty.down.sql b/deploy/migrations/clickhouse/023_cannon_proposer_duty.down.sql deleted file mode 100644 index 1b68c6a6..00000000 --- a/deploy/migrations/clickhouse/023_cannon_proposer_duty.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_proposer_duty on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_proposer_duty_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/023_cannon_proposer_duty.up.sql b/deploy/migrations/clickhouse/023_cannon_proposer_duty.up.sql deleted file mode 100644 index 4e722eee..00000000 --- a/deploy/migrations/clickhouse/023_cannon_proposer_duty.up.sql +++ /dev/null @@ -1,73 +0,0 @@ -CREATE TABLE default.canonical_beacon_proposer_duty_local on cluster '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - proposer_validator_index UInt32 CODEC(ZSTD(1)), - proposer_pubkey String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a proposer duty from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN proposer_validator_index 'The validator index from the proposer duty payload', -COMMENT COLUMN proposer_pubkey 'The BLS public key of the validator from the proposer duty payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_proposer_duty on cluster '{cluster}' AS canonical_beacon_proposer_duty_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_proposer_duty_local, rand()); diff --git a/deploy/migrations/clickhouse/024_cannon_elaborated_attestation.down.sql b/deploy/migrations/clickhouse/024_cannon_elaborated_attestation.down.sql deleted file mode 100644 index 859cc835..00000000 --- a/deploy/migrations/clickhouse/024_cannon_elaborated_attestation.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_elaborated_attestation on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_elaborated_attestation_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/024_cannon_elaborated_attestation.up.sql b/deploy/migrations/clickhouse/024_cannon_elaborated_attestation.up.sql deleted file mode 100644 index ec525549..00000000 --- a/deploy/migrations/clickhouse/024_cannon_elaborated_attestation.up.sql +++ /dev/null @@ -1,96 +0,0 @@ -CREATE TABLE default.canonical_beacon_elaborated_attestation_local on cluster '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - block_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - block_slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - block_epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - position_in_block UInt32 Codec(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - validators Array(UInt32) Codec(ZSTD(1)), - committee_index LowCardinality(String), - beacon_block_root FixedString(66) Codec(ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - source_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - source_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source_root FixedString(66) Codec(ZSTD(1)), - target_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - target_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - target_root FixedString(66) Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains elaborated attestations from beacon blocks.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the elaborated attestation from a beacon node', -COMMENT COLUMN block_slot 'The slot number of the block containing the attestation', -COMMENT COLUMN block_slot_start_date_time 'The wall clock time when the block slot started', -COMMENT COLUMN block_epoch 'The epoch number of the block containing the attestation', -COMMENT COLUMN block_epoch_start_date_time 'The wall clock time when the block epoch started', -COMMENT COLUMN position_in_block 'The position of the attestation in the block', -COMMENT COLUMN block_root 'The root of the block containing the attestation', -COMMENT COLUMN validators 'Array of validator indices participating in the attestation', -COMMENT COLUMN committee_index 'The index of the committee making the attestation', -COMMENT COLUMN beacon_block_root 'The root of the beacon block being attested to', -COMMENT COLUMN slot 'The slot number being attested to', -COMMENT COLUMN source_epoch 'The source epoch referenced in the attestation', -COMMENT COLUMN source_epoch_start_date_time 'The wall clock time when the source epoch started', -COMMENT COLUMN source_root 'The root of the source checkpoint in the attestation', -COMMENT COLUMN target_epoch 'The target epoch referenced in the attestation', -COMMENT COLUMN target_epoch_start_date_time 'The wall clock time when the target epoch started', -COMMENT COLUMN target_root 'The root of the target checkpoint in the attestation', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_elaborated_attestation on cluster '{cluster}' AS canonical_beacon_elaborated_attestation_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_elaborated_attestation_local, rand()); diff --git a/deploy/migrations/clickhouse/025_rm_beacon_attestation_signature.down.sql b/deploy/migrations/clickhouse/025_rm_beacon_attestation_signature.down.sql deleted file mode 100644 index 79e942cc..00000000 --- a/deploy/migrations/clickhouse/025_rm_beacon_attestation_signature.down.sql +++ /dev/null @@ -1,11 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' -ADD COLUMN signature String Codec(ZSTD(1)) AFTER attesting_validator_committee_index; - -ALTER TABLE default.beacon_api_eth_v1_events_attestation ON CLUSTER '{cluster}' -ADD COLUMN signature String Codec(ZSTD(1)) AFTER attesting_validator_committee_index; - -ALTER TABLE default.beacon_p2p_attestation_local ON CLUSTER '{cluster}' -ADD COLUMN signature String Codec(ZSTD(1)) AFTER attesting_validator_committee_index; - -ALTER TABLE default.beacon_p2p_attestation ON CLUSTER '{cluster}' -ADD COLUMN signature String Codec(ZSTD(1)) AFTER attesting_validator_committee_index; diff --git a/deploy/migrations/clickhouse/025_rm_beacon_attestation_signature.up.sql b/deploy/migrations/clickhouse/025_rm_beacon_attestation_signature.up.sql deleted file mode 100644 index 6415862e..00000000 --- a/deploy/migrations/clickhouse/025_rm_beacon_attestation_signature.up.sql +++ /dev/null @@ -1,11 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v1_events_attestation ON CLUSTER '{cluster}' -DROP COLUMN signature; - -ALTER TABLE default.beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' -DROP COLUMN signature; - -ALTER TABLE default.beacon_p2p_attestation ON CLUSTER '{cluster}' -DROP COLUMN signature; - -ALTER TABLE default.beacon_p2p_attestation_local ON CLUSTER '{cluster}' -DROP COLUMN signature; diff --git a/deploy/migrations/clickhouse/026_dbt_model_metadata.down.sql b/deploy/migrations/clickhouse/026_dbt_model_metadata.down.sql deleted file mode 100644 index a72b57ab..00000000 --- a/deploy/migrations/clickhouse/026_dbt_model_metadata.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS model_metadata on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS model_metadata_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/026_dbt_model_metadata.up.sql b/deploy/migrations/clickhouse/026_dbt_model_metadata.up.sql deleted file mode 100644 index 2f5d64d4..00000000 --- a/deploy/migrations/clickhouse/026_dbt_model_metadata.up.sql +++ /dev/null @@ -1,17 +0,0 @@ -CREATE TABLE dbt.model_metadata_local on cluster '{cluster}' -( - model LowCardinality(String), - updated_date_time DateTime Codec(ZSTD(1)), - last_run_date_time DateTime Codec(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY model -ORDER BY (model); - -ALTER TABLE dbt.model_metadata_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Stores metadata about the dbt model run', -COMMENT COLUMN model 'Source of the data that was imported', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN last_run_date_time 'The end date the model was last run with'; - -CREATE TABLE dbt.model_metadata on cluster '{cluster}' AS dbt.model_metadata_local -ENGINE = Distributed('{cluster}', dbt, model_metadata_local, cityHash64(model)); diff --git a/deploy/migrations/clickhouse/027_blob_txn.down.sql b/deploy/migrations/clickhouse/027_blob_txn.down.sql deleted file mode 100644 index 42d1a3d4..00000000 --- a/deploy/migrations/clickhouse/027_blob_txn.down.sql +++ /dev/null @@ -1,169 +0,0 @@ -ALTER TABLE mempool_transaction_local on cluster '{cluster}' - DROP COLUMN gas_tip_cap, - DROP COLUMN gas_fee_cap, - DROP COLUMN blob_gas, - DROP COLUMN blob_gas_fee_cap, - DROP COLUMN blob_hashes, - DROP COLUMN blob_sidecars_size, - DROP COLUMN blob_sidecars_empty_size; - -ALTER TABLE mempool_transaction on cluster '{cluster}' - DROP COLUMN gas_tip_cap, - DROP COLUMN gas_fee_cap, - DROP COLUMN blob_gas, - DROP COLUMN blob_gas_fee_cap, - DROP COLUMN blob_hashes, - DROP COLUMN blob_sidecars_size, - DROP COLUMN blob_sidecars_empty_size; - -ALTER TABLE canonical_beacon_block_execution_transaction_local on cluster '{cluster}' - DROP COLUMN gas_tip_cap, - DROP COLUMN gas_fee_cap, - DROP COLUMN blob_gas, - DROP COLUMN blob_gas_fee_cap, - DROP COLUMN blob_hashes, - DROP COLUMN blob_sidecars_size, - DROP COLUMN blob_sidecars_empty_size; - -ALTER TABLE canonical_beacon_block_execution_transaction on cluster '{cluster}' - DROP COLUMN gas_tip_cap, - DROP COLUMN gas_fee_cap, - DROP COLUMN blob_gas, - DROP COLUMN blob_gas_fee_cap, - DROP COLUMN blob_hashes, - DROP COLUMN blob_sidecars_size, - DROP COLUMN blob_sidecars_empty_size; - -ALTER TABLE canonical_beacon_blob_sidecar_local on cluster '{cluster}' - DROP COLUMN blob_empty_size; - -ALTER TABLE canonical_beacon_blob_sidecar on cluster '{cluster}' - DROP COLUMN blob_empty_size; - -ALTER TABLE block_native_mempool_transaction_local on cluster '{cluster}' -COMMENT COLUMN unique_key '', -COMMENT COLUMN updated_date_time '', -COMMENT COLUMN detecttime '', -COMMENT COLUMN hash '', -COMMENT COLUMN status '', -COMMENT COLUMN region '', -COMMENT COLUMN reorg '', -COMMENT COLUMN replace '', -COMMENT COLUMN curblocknumber '', -COMMENT COLUMN failurereason '', -COMMENT COLUMN blockspending '', -COMMENT COLUMN timepending '', -COMMENT COLUMN nonce '', -COMMENT COLUMN gas '', -COMMENT COLUMN gasprice '', -COMMENT COLUMN value '', -COMMENT COLUMN toaddress '', -COMMENT COLUMN fromaddress '', -COMMENT COLUMN datasize '', -COMMENT COLUMN data4bytes '', -COMMENT COLUMN network '', -COMMENT COLUMN type '', -COMMENT COLUMN maxpriorityfeepergas '', -COMMENT COLUMN maxfeepergas '', -COMMENT COLUMN basefeepergas '', -COMMENT COLUMN dropreason '', -COMMENT COLUMN rejectionreason '', -COMMENT COLUMN stuck '', -COMMENT COLUMN gasused ''; - -ALTER TABLE block_native_mempool_transaction on cluster '{cluster}' -COMMENT COLUMN unique_key '', -COMMENT COLUMN updated_date_time '', -COMMENT COLUMN detecttime '', -COMMENT COLUMN hash '', -COMMENT COLUMN status '', -COMMENT COLUMN region '', -COMMENT COLUMN reorg '', -COMMENT COLUMN replace '', -COMMENT COLUMN curblocknumber '', -COMMENT COLUMN failurereason '', -COMMENT COLUMN blockspending '', -COMMENT COLUMN timepending '', -COMMENT COLUMN nonce '', -COMMENT COLUMN gas '', -COMMENT COLUMN gasprice '', -COMMENT COLUMN value '', -COMMENT COLUMN toaddress '', -COMMENT COLUMN fromaddress '', -COMMENT COLUMN datasize '', -COMMENT COLUMN data4bytes '', -COMMENT COLUMN network '', -COMMENT COLUMN type '', -COMMENT COLUMN maxpriorityfeepergas '', -COMMENT COLUMN maxfeepergas '', -COMMENT COLUMN basefeepergas '', -COMMENT COLUMN dropreason '', -COMMENT COLUMN rejectionreason '', -COMMENT COLUMN stuck '', -COMMENT COLUMN gasused ''; - -ALTER TABLE canonical_beacon_proposer_duty_local on cluster '{cluster}' -COMMENT COLUMN unique_key '', -COMMENT COLUMN updated_date_time '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN proposer_validator_index '', -COMMENT COLUMN proposer_pubkey '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE canonical_beacon_proposer_duty on cluster '{cluster}' -COMMENT COLUMN unique_key '', -COMMENT COLUMN updated_date_time '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN proposer_validator_index '', -COMMENT COLUMN proposer_pubkey '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; diff --git a/deploy/migrations/clickhouse/027_blob_txn.up.sql b/deploy/migrations/clickhouse/027_blob_txn.up.sql deleted file mode 100644 index 06183d01..00000000 --- a/deploy/migrations/clickhouse/027_blob_txn.up.sql +++ /dev/null @@ -1,199 +0,0 @@ -ALTER TABLE mempool_transaction_local on cluster '{cluster}' -ADD COLUMN gas_tip_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER gas, -ADD COLUMN gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER gas_tip_cap, -ADD COLUMN blob_gas Nullable(UInt64) Codec(ZSTD(1)) AFTER call_data_size, -ADD COLUMN blob_gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER blob_gas, -ADD COLUMN blob_hashes Array(String) Codec(ZSTD(1)) AFTER blob_gas_fee_cap, -ADD COLUMN blob_sidecars_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_hashes, -ADD COLUMN blob_sidecars_empty_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_sidecars_size, -COMMENT COLUMN gas_tip_cap 'The priority fee (tip) the user has set for the transaction', -COMMENT COLUMN gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_gas 'The maximum gas provided for the blob transaction execution', -COMMENT COLUMN blob_gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_hashes 'The hashes of the blob commitments for blob transactions', -COMMENT COLUMN blob_sidecars_size 'The total size of the sidecars for blob transactions in bytes', -COMMENT COLUMN blob_sidecars_empty_size 'The total empty size of the sidecars for blob transactions in bytes'; - -ALTER TABLE mempool_transaction on cluster '{cluster}' -ADD COLUMN gas_tip_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER gas, -ADD COLUMN gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER gas_tip_cap, -ADD COLUMN blob_gas Nullable(UInt64) Codec(ZSTD(1)) AFTER call_data_size, -ADD COLUMN blob_gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER blob_gas, -ADD COLUMN blob_hashes Array(String) Codec(ZSTD(1)) AFTER blob_gas_fee_cap, -ADD COLUMN blob_sidecars_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_hashes, -ADD COLUMN blob_sidecars_empty_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_sidecars_size, -COMMENT COLUMN gas_tip_cap 'The priority fee (tip) the user has set for the transaction', -COMMENT COLUMN gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_gas 'The maximum gas provided for the blob transaction execution', -COMMENT COLUMN blob_gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_hashes 'The hashes of the blob commitments for blob transactions', -COMMENT COLUMN blob_sidecars_size 'The total size of the sidecars for blob transactions in bytes', -COMMENT COLUMN blob_sidecars_empty_size 'The total empty size of the sidecars for blob transactions in bytes'; - -ALTER TABLE canonical_beacon_block_execution_transaction_local on cluster '{cluster}' -ADD COLUMN gas_tip_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER gas, -ADD COLUMN gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER gas_tip_cap, -ADD COLUMN blob_gas Nullable(UInt64) Codec(ZSTD(1)) AFTER call_data_size, -ADD COLUMN blob_gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER blob_gas, -ADD COLUMN blob_hashes Array(String) Codec(ZSTD(1)) AFTER blob_gas_fee_cap, -ADD COLUMN blob_sidecars_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_hashes, -ADD COLUMN blob_sidecars_empty_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_sidecars_size, -COMMENT COLUMN gas_tip_cap 'The priority fee (tip) the user has set for the transaction', -COMMENT COLUMN gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_gas 'The maximum gas provided for the blob transaction execution', -COMMENT COLUMN blob_gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_hashes 'The hashes of the blob commitments for blob transactions', -COMMENT COLUMN blob_sidecars_size 'The total size of the sidecars for blob transactions in bytes', -COMMENT COLUMN blob_sidecars_empty_size 'The total empty size of the sidecars for blob transactions in bytes'; - -ALTER TABLE canonical_beacon_block_execution_transaction on cluster '{cluster}' -ADD COLUMN gas_tip_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER gas, -ADD COLUMN gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER gas_tip_cap, -ADD COLUMN blob_gas Nullable(UInt64) Codec(ZSTD(1)) AFTER call_data_size, -ADD COLUMN blob_gas_fee_cap Nullable(UInt128) Codec(ZSTD(1)) AFTER blob_gas, -ADD COLUMN blob_hashes Array(String) Codec(ZSTD(1)) AFTER blob_gas_fee_cap, -ADD COLUMN blob_sidecars_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_hashes, -ADD COLUMN blob_sidecars_empty_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_sidecars_size, -COMMENT COLUMN gas_tip_cap 'The priority fee (tip) the user has set for the transaction', -COMMENT COLUMN gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_gas 'The maximum gas provided for the blob transaction execution', -COMMENT COLUMN blob_gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_hashes 'The hashes of the blob commitments for blob transactions', -COMMENT COLUMN blob_sidecars_size 'The total size of the sidecars for blob transactions in bytes', -COMMENT COLUMN blob_sidecars_empty_size 'The total empty size of the sidecars for blob transactions in bytes'; - -ALTER TABLE canonical_beacon_blob_sidecar_local on cluster '{cluster}' -ADD COLUMN blob_empty_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_size, -COMMENT COLUMN blob_empty_size 'The total empty size of the blob in bytes'; - -ALTER TABLE canonical_beacon_blob_sidecar on cluster '{cluster}' -ADD COLUMN blob_empty_size Nullable(UInt32) Codec(ZSTD(1)) AFTER blob_size, -COMMENT COLUMN blob_empty_size 'The total empty size of the blob in bytes'; - -ALTER TABLE block_native_mempool_transaction_local on cluster '{cluster}' -COMMENT COLUMN unique_key 'Unique key for the row', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN detecttime 'Timestamp that the transaction was detected in mempool', -COMMENT COLUMN hash 'Unique identifier hash for a given transaction', -COMMENT COLUMN status 'Status of the transaction', -COMMENT COLUMN region 'The geographic region for the node that detected the transaction', -COMMENT COLUMN reorg 'If there was a reorg, refers to the blockhash of the reorg', -COMMENT COLUMN replace 'If the transaction was replaced (speedup/cancel), the transaction hash of the replacement', -COMMENT COLUMN curblocknumber 'The block number the event was detected in', -COMMENT COLUMN failurereason 'If a transaction failed, this field provides contextual information', -COMMENT COLUMN blockspending 'If a transaction was finalized (confirmed, failed), this refers to the number of blocks that the transaction was waiting to get on-chain', -COMMENT COLUMN timepending 'If a transaction was finalized (confirmed, failed), this refers to the time in milliseconds that the transaction was waiting to get on-chain', -COMMENT COLUMN nonce 'A unique number which counts the number of transactions sent from a given address', -COMMENT COLUMN gas 'The maximum number of gas units allowed for the transaction', -COMMENT COLUMN gasprice 'The price offered to the miner/validator per unit of gas. Denominated in wei', -COMMENT COLUMN value 'The amount of ETH transferred or sent to contract. Denominated in wei', -COMMENT COLUMN toaddress 'The destination of a given transaction', -COMMENT COLUMN fromaddress 'The source/initiator of a given transaction', -COMMENT COLUMN datasize 'The size of the call data of the transaction in bytes', -COMMENT COLUMN data4bytes 'The first 4 bytes of the call data of the transaction', -COMMENT COLUMN network 'The specific Ethereum network used', -COMMENT COLUMN type '"Post EIP-1559, this indicates how the gas parameters are submitted to the network: - type 0 - legacy - type 1 - usage of access lists according to EIP-2930 - type 2 - using maxpriorityfeepergas and maxfeepergas"', -COMMENT COLUMN maxpriorityfeepergas 'The maximum value for a tip offered to the miner/validator per unit of gas. The actual tip paid can be lower if (maxfee - basefee) < maxpriorityfee. Denominated in wei', -COMMENT COLUMN maxfeepergas 'The maximum value for the transaction fee (including basefee and tip) offered to the miner/validator per unit of gas. Denominated in wei', -COMMENT COLUMN basefeepergas 'The fee per unit of gas paid and burned for the curblocknumber. This fee is algorithmically determined. Denominated in wei', -COMMENT COLUMN dropreason 'If the transaction was dropped from the mempool, this describes the contextual reason for the drop', -COMMENT COLUMN rejectionreason 'If the transaction was rejected from the mempool, this describes the contextual reason for the rejection', -COMMENT COLUMN stuck 'A transaction was detected in the queued area of the mempool and is not eligible for inclusion in a block', -COMMENT COLUMN gasused 'If the transaction was published on-chain, this value indicates the amount of gas that was actually consumed. Denominated in wei'; - -ALTER TABLE block_native_mempool_transaction on cluster '{cluster}' -COMMENT COLUMN unique_key 'Unique key for the row', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN detecttime 'Timestamp that the transaction was detected in mempool', -COMMENT COLUMN hash 'Unique identifier hash for a given transaction', -COMMENT COLUMN status 'Status of the transaction', -COMMENT COLUMN region 'The geographic region for the node that detected the transaction', -COMMENT COLUMN reorg 'If there was a reorg, refers to the blockhash of the reorg', -COMMENT COLUMN replace 'If the transaction was replaced (speedup/cancel), the transaction hash of the replacement', -COMMENT COLUMN curblocknumber 'The block number the event was detected in', -COMMENT COLUMN failurereason 'If a transaction failed, this field provides contextual information', -COMMENT COLUMN blockspending 'If a transaction was finalized (confirmed, failed), this refers to the number of blocks that the transaction was waiting to get on-chain', -COMMENT COLUMN timepending 'If a transaction was finalized (confirmed, failed), this refers to the time in milliseconds that the transaction was waiting to get on-chain', -COMMENT COLUMN nonce 'A unique number which counts the number of transactions sent from a given address', -COMMENT COLUMN gas 'The maximum number of gas units allowed for the transaction', -COMMENT COLUMN gasprice 'The price offered to the miner/validator per unit of gas. Denominated in wei', -COMMENT COLUMN value 'The amount of ETH transferred or sent to contract. Denominated in wei', -COMMENT COLUMN toaddress 'The destination of a given transaction', -COMMENT COLUMN fromaddress 'The source/initiator of a given transaction', -COMMENT COLUMN datasize 'The size of the call data of the transaction in bytes', -COMMENT COLUMN data4bytes 'The first 4 bytes of the call data of the transaction', -COMMENT COLUMN network 'The specific Ethereum network used', -COMMENT COLUMN type '"Post EIP-1559, this indicates how the gas parameters are submitted to the network: - type 0 - legacy - type 1 - usage of access lists according to EIP-2930 - type 2 - using maxpriorityfeepergas and maxfeepergas"', -COMMENT COLUMN maxpriorityfeepergas 'The maximum value for a tip offered to the miner/validator per unit of gas. The actual tip paid can be lower if (maxfee - basefee) < maxpriorityfee. Denominated in wei', -COMMENT COLUMN maxfeepergas 'The maximum value for the transaction fee (including basefee and tip) offered to the miner/validator per unit of gas. Denominated in wei', -COMMENT COLUMN basefeepergas 'The fee per unit of gas paid and burned for the curblocknumber. This fee is algorithmically determined. Denominated in wei', -COMMENT COLUMN dropreason 'If the transaction was dropped from the mempool, this describes the contextual reason for the drop', -COMMENT COLUMN rejectionreason 'If the transaction was rejected from the mempool, this describes the contextual reason for the rejection', -COMMENT COLUMN stuck 'A transaction was detected in the queued area of the mempool and is not eligible for inclusion in a block', -COMMENT COLUMN gasused 'If the transaction was published on-chain, this value indicates the amount of gas that was actually consumed. Denominated in wei'; - -ALTER TABLE canonical_beacon_proposer_duty_local on cluster '{cluster}' -COMMENT COLUMN unique_key 'Unique key for the row', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the proposer duty information from a beacon node', -COMMENT COLUMN slot 'The slot number for which the proposer duty is assigned', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number containing the slot', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN proposer_validator_index 'The validator index of the proposer for the slot', -COMMENT COLUMN proposer_pubkey 'The public key of the validator proposer', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the even'; - -ALTER TABLE canonical_beacon_proposer_duty on cluster '{cluster}' -COMMENT COLUMN unique_key 'Unique key for the row', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the proposer duty information from a beacon node', -COMMENT COLUMN slot 'The slot number for which the proposer duty is assigned', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number containing the slot', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN proposer_validator_index 'The validator index of the proposer for the slot', -COMMENT COLUMN proposer_pubkey 'The public key of the validator proposer', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the even'; diff --git a/deploy/migrations/clickhouse/028_comments.down.sql b/deploy/migrations/clickhouse/028_comments.down.sql deleted file mode 100644 index b89fa09c..00000000 --- a/deploy/migrations/clickhouse/028_comments.down.sql +++ /dev/null @@ -1,75 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN committee_index '', -COMMENT COLUMN validators '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' -MODIFY COMMENT '', -COMMENT COLUMN event_date_time '', -COMMENT COLUMN slot '', -COMMENT COLUMN slot_start_date_time '', -COMMENT COLUMN committee_index '', -COMMENT COLUMN validators '', -COMMENT COLUMN epoch '', -COMMENT COLUMN epoch_start_date_time '', -COMMENT COLUMN meta_client_name '', -COMMENT COLUMN meta_client_id '', -COMMENT COLUMN meta_client_version '', -COMMENT COLUMN meta_client_implementation '', -COMMENT COLUMN meta_client_os '', -COMMENT COLUMN meta_client_ip '', -COMMENT COLUMN meta_client_geo_city '', -COMMENT COLUMN meta_client_geo_country '', -COMMENT COLUMN meta_client_geo_country_code '', -COMMENT COLUMN meta_client_geo_continent_code '', -COMMENT COLUMN meta_client_geo_longitude '', -COMMENT COLUMN meta_client_geo_latitude '', -COMMENT COLUMN meta_client_geo_autonomous_system_number '', -COMMENT COLUMN meta_client_geo_autonomous_system_organization '', -COMMENT COLUMN meta_network_id '', -COMMENT COLUMN meta_network_name '', -COMMENT COLUMN meta_consensus_version '', -COMMENT COLUMN meta_consensus_version_major '', -COMMENT COLUMN meta_consensus_version_minor '', -COMMENT COLUMN meta_consensus_version_patch '', -COMMENT COLUMN meta_consensus_implementation '', -COMMENT COLUMN meta_labels ''; - -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT ''; - -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT ''; - -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' -MODIFY COMMENT ''; - -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' -MODIFY COMMENT ''; diff --git a/deploy/migrations/clickhouse/028_comments.up.sql b/deploy/migrations/clickhouse/028_comments.up.sql deleted file mode 100644 index a2565af0..00000000 --- a/deploy/migrations/clickhouse/028_comments.up.sql +++ /dev/null @@ -1,75 +0,0 @@ -ALTER TABLE default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API committee payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN committee_index 'The committee index in the beacon API committee payload', -COMMENT COLUMN validators 'The validator indices in the beacon API committee payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API committee payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API committee payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN committee_index 'The committee index in the beacon API committee payload', -COMMENT COLUMN validators 'The validator indices in the beacon API committee payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API committee payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.'; - -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.'; - -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/validator/attestation_data data from each sentry client attached to a beacon node.'; - -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/validator/attestation_data data from each sentry client attached to a beacon node.'; diff --git a/deploy/migrations/clickhouse/030_fix_replacing_tables_sharding_keys.down.sql b/deploy/migrations/clickhouse/030_fix_replacing_tables_sharding_keys.down.sql deleted file mode 100644 index a4c3d9a9..00000000 --- a/deploy/migrations/clickhouse/030_fix_replacing_tables_sharding_keys.down.sql +++ /dev/null @@ -1,59 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_block on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block on cluster '{cluster}' AS canonical_beacon_block_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_block_proposer_slashing on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_proposer_slashing on cluster '{cluster}' AS canonical_beacon_block_proposer_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_proposer_slashing_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_block_attester_slashing on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_attester_slashing on cluster '{cluster}' AS canonical_beacon_block_attester_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_attester_slashing_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' AS canonical_beacon_block_bls_to_execution_change_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_bls_to_execution_change_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_block_execution_transaction on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_execution_transaction on cluster '{cluster}' AS canonical_beacon_block_execution_transaction_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_execution_transaction_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_block_voluntary_exit on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_voluntary_exit on cluster '{cluster}' AS canonical_beacon_block_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_voluntary_exit_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_block_deposit on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_deposit on cluster '{cluster}' AS canonical_beacon_block_deposit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_deposit_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_block_withdrawal on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_withdrawal on cluster '{cluster}' AS canonical_beacon_block_withdrawal_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_withdrawal_local, rand()); - -DROP TABLE IF EXISTS beacon_block_classification on cluster '{cluster}' SYNC; -CREATE TABLE beacon_block_classification on cluster '{cluster}' AS beacon_block_classification_local -ENGINE = Distributed('{cluster}', default, beacon_block_classification_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_blob_sidecar on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_blob_sidecar on cluster '{cluster}' AS canonical_beacon_blob_sidecar_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_blob_sidecar_local, rand()); - -DROP TABLE IF EXISTS mempool_dumpster_transaction on cluster '{cluster}' SYNC; -CREATE TABLE mempool_dumpster_transaction on cluster '{cluster}' AS mempool_dumpster_transaction_local -ENGINE = Distributed('{cluster}', default, mempool_dumpster_transaction_local, rand()); - -DROP TABLE IF EXISTS block_native_mempool_transaction on cluster '{cluster}' SYNC; -CREATE TABLE block_native_mempool_transaction on cluster '{cluster}' AS block_native_mempool_transaction_local -ENGINE = Distributed('{cluster}', default, block_native_mempool_transaction_local, rand()); - -DROP TABLE IF EXISTS beacon_p2p_attestation on cluster '{cluster}' SYNC; -CREATE TABLE beacon_p2p_attestation on cluster '{cluster}' AS beacon_p2p_attestation_local -ENGINE = Distributed('{cluster}', default, beacon_p2p_attestation_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_proposer_duty on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_proposer_duty on cluster '{cluster}' AS canonical_beacon_proposer_duty_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_proposer_duty_local, rand()); - -DROP TABLE IF EXISTS canonical_beacon_elaborated_attestation on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_elaborated_attestation on cluster '{cluster}' AS canonical_beacon_elaborated_attestation_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_elaborated_attestation_local, rand()); diff --git a/deploy/migrations/clickhouse/030_fix_replacing_tables_sharding_keys.up.sql b/deploy/migrations/clickhouse/030_fix_replacing_tables_sharding_keys.up.sql deleted file mode 100644 index 5760bf06..00000000 --- a/deploy/migrations/clickhouse/030_fix_replacing_tables_sharding_keys.up.sql +++ /dev/null @@ -1,59 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_block on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block on cluster '{cluster}' AS canonical_beacon_block_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_local, cityHash64(slot_start_date_time, meta_network_name)); - -DROP TABLE IF EXISTS canonical_beacon_block_proposer_slashing on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_proposer_slashing on cluster '{cluster}' AS canonical_beacon_block_proposer_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_proposer_slashing_local, unique_key); - -DROP TABLE IF EXISTS canonical_beacon_block_attester_slashing on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_attester_slashing on cluster '{cluster}' AS canonical_beacon_block_attester_slashing_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_attester_slashing_local, unique_key); - -DROP TABLE IF EXISTS canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_bls_to_execution_change on cluster '{cluster}' AS canonical_beacon_block_bls_to_execution_change_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_bls_to_execution_change_local, unique_key); - -DROP TABLE IF EXISTS canonical_beacon_block_execution_transaction on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_execution_transaction on cluster '{cluster}' AS canonical_beacon_block_execution_transaction_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_execution_transaction_local, unique_key); - -DROP TABLE IF EXISTS canonical_beacon_block_voluntary_exit on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_voluntary_exit on cluster '{cluster}' AS canonical_beacon_block_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_voluntary_exit_local, unique_key); - -DROP TABLE IF EXISTS canonical_beacon_block_deposit on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_deposit on cluster '{cluster}' AS canonical_beacon_block_deposit_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_deposit_local, unique_key); - -DROP TABLE IF EXISTS canonical_beacon_block_withdrawal on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_block_withdrawal on cluster '{cluster}' AS canonical_beacon_block_withdrawal_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_block_withdrawal_local, unique_key); - -DROP TABLE IF EXISTS beacon_block_classification on cluster '{cluster}' SYNC; -CREATE TABLE beacon_block_classification on cluster '{cluster}' AS beacon_block_classification_local -ENGINE = Distributed('{cluster}', default, beacon_block_classification_local, cityHash64(slot_start_date_time, meta_network_name)); - -DROP TABLE IF EXISTS canonical_beacon_blob_sidecar on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_blob_sidecar on cluster '{cluster}' AS canonical_beacon_blob_sidecar_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_blob_sidecar_local, unique_key); - -DROP TABLE IF EXISTS mempool_dumpster_transaction on cluster '{cluster}' SYNC; -CREATE TABLE mempool_dumpster_transaction on cluster '{cluster}' AS mempool_dumpster_transaction_local -ENGINE = Distributed('{cluster}', default, mempool_dumpster_transaction_local, unique_key); - -DROP TABLE IF EXISTS block_native_mempool_transaction on cluster '{cluster}' SYNC; -CREATE TABLE block_native_mempool_transaction on cluster '{cluster}' AS block_native_mempool_transaction_local -ENGINE = Distributed('{cluster}', default, block_native_mempool_transaction_local, unique_key); - -DROP TABLE IF EXISTS beacon_p2p_attestation on cluster '{cluster}' SYNC; -CREATE TABLE beacon_p2p_attestation on cluster '{cluster}' AS beacon_p2p_attestation_local -ENGINE = Distributed('{cluster}', default, beacon_p2p_attestation_local, unique_key); - -DROP TABLE IF EXISTS canonical_beacon_proposer_duty on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_proposer_duty on cluster '{cluster}' AS canonical_beacon_proposer_duty_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_proposer_duty_local, unique_key); - -DROP TABLE IF EXISTS canonical_beacon_elaborated_attestation on cluster '{cluster}' SYNC; -CREATE TABLE canonical_beacon_elaborated_attestation on cluster '{cluster}' AS canonical_beacon_elaborated_attestation_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_elaborated_attestation_local, unique_key); diff --git a/deploy/migrations/clickhouse/031_v1_proposer_duties.down.sql b/deploy/migrations/clickhouse/031_v1_proposer_duties.down.sql deleted file mode 100644 index ff1e611e..00000000 --- a/deploy/migrations/clickhouse/031_v1_proposer_duties.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/031_v1_proposer_duties.up.sql b/deploy/migrations/clickhouse/031_v1_proposer_duties.up.sql deleted file mode 100644 index 29211674..00000000 --- a/deploy/migrations/clickhouse/031_v1_proposer_duties.up.sql +++ /dev/null @@ -1,73 +0,0 @@ -CREATE TABLE default.beacon_api_eth_v1_proposer_duty_local on cluster '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - proposer_validator_index UInt32 CODEC(ZSTD(1)), - proposer_pubkey String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name); - -ALTER TABLE default.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a proposer duty from a beacon block.', -COMMENT COLUMN unique_key 'Unique key for the row generated from seahash', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN proposer_validator_index 'The validator index from the proposer duty payload', -COMMENT COLUMN proposer_pubkey 'The BLS public key of the validator from the proposer duty payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v1_proposer_duty on cluster '{cluster}' AS beacon_api_eth_v1_proposer_duty_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_proposer_duty_local, rand()); diff --git a/deploy/migrations/clickhouse/032_libp2p_trace.down.sql b/deploy/migrations/clickhouse/032_libp2p_trace.down.sql deleted file mode 100644 index 48f76d08..00000000 --- a/deploy/migrations/clickhouse/032_libp2p_trace.down.sql +++ /dev/null @@ -1,47 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_recv_rpc ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_recv_rpc_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_send_rpc ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_send_rpc_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_drop_rpc ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_drop_rpc_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_rpc_meta_message ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_rpc_meta_message_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_rpc_meta_subscription ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_rpc_meta_subscription_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_ihave ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_ihave_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_iwant ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_iwant_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_graft ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_graft_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_prune ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_prune_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_recv_rpc ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_recv_rpc_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_join ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_join_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_add_peer ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_add_peer_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_remove_peer ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_remove_peer_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_connected ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_connected_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_disconnected ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_disconnected_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS default.libp2p_peer ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_peer_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/032_libp2p_trace.up.sql b/deploy/migrations/clickhouse/032_libp2p_trace.up.sql deleted file mode 100644 index ba0b13bf..00000000 --- a/deploy/migrations/clickhouse/032_libp2p_trace.up.sql +++ /dev/null @@ -1,904 +0,0 @@ --- Creating local and distributed tables for libp2p_peer -CREATE TABLE libp2p_peer_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - peer_id String CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -ORDER BY (unique_key); - -ALTER TABLE libp2p_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the original peer id of a seahashed peer_id + meta_network_name, commonly seen in other tables as the field peer_id_unique_key', -COMMENT COLUMN unique_key 'Unique identifier for each record, seahash of peer_id + meta_network_name', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN peer_id 'Peer ID', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_peer ON CLUSTER '{cluster}' AS libp2p_peer_local -ENGINE = Distributed('{cluster}', default, libp2p_peer_local, unique_key); - --- Creating local and distributed tables for libp2p_add_peer -CREATE TABLE libp2p_add_peer_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - peer_id_unique_key Int64, - protocol LowCardinality(String), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_add_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the peers added to the libp2p client.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer', -COMMENT COLUMN protocol 'Protocol used by the peer', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_add_peer ON CLUSTER '{cluster}' AS libp2p_add_peer_local -ENGINE = Distributed('{cluster}', default, libp2p_add_peer_local, unique_key); - --- Creating local and distributed tables for libp2p_remove_peer -CREATE TABLE libp2p_remove_peer_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_remove_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the peers removed from the libp2p client.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_remove_peer ON CLUSTER '{cluster}' AS libp2p_remove_peer_local -ENGINE = Distributed('{cluster}', default, libp2p_remove_peer_local, unique_key); - --- Creating tables for RPC meta data with ReplicatedReplacingMergeTree and Distributed engines -CREATE TABLE libp2p_rpc_meta_message_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - control_index Int32 CODEC(DoubleDelta, ZSTD(1)), - rpc_meta_unique_key Int64, - message_id String CODEC(ZSTD(1)), - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, control_index, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_rpc_meta_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC meta messages from the peer', -COMMENT COLUMN unique_key 'Unique identifier for each RPC message record', -COMMENT COLUMN event_date_time 'Timestamp of the RPC event', -COMMENT COLUMN updated_date_time 'Timestamp when the RPC message record was last updated', -COMMENT COLUMN control_index 'Position in the RPC meta message array', -COMMENT COLUMN rpc_meta_unique_key 'Unique key associated with the RPC metadata', -COMMENT COLUMN message_id 'Identifier of the message', -COMMENT COLUMN topic_layer 'Layer of the topic', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding of the topic', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer involved in the RPC', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_rpc_meta_message ON CLUSTER '{cluster}' AS libp2p_rpc_meta_message_local -ENGINE = Distributed('{cluster}', default, libp2p_rpc_meta_message_local, unique_key); - -CREATE TABLE libp2p_rpc_meta_subscription_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - control_index Int32 CODEC(DoubleDelta, ZSTD(1)), - rpc_meta_unique_key Int64, - subscribe Bool, - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, control_index, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_rpc_meta_subscription_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC subscriptions from the peer.', -COMMENT COLUMN unique_key 'Unique identifier for each RPC subscription record', -COMMENT COLUMN updated_date_time 'Timestamp when the RPC subscription record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the RPC subscription event', -COMMENT COLUMN control_index 'Position in the RPC meta subscription array', -COMMENT COLUMN rpc_meta_unique_key 'Unique key associated with the RPC subscription metadata', -COMMENT COLUMN subscribe 'Boolean indicating if it is a subscription or unsubscription', -COMMENT COLUMN topic_layer 'Layer of the topic', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding of the topic', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer involved in the subscription', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_rpc_meta_subscription ON CLUSTER '{cluster}' AS libp2p_rpc_meta_subscription_local -ENGINE = Distributed('{cluster}', default, libp2p_rpc_meta_subscription_local, unique_key); - -CREATE TABLE libp2p_rpc_meta_control_ihave_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - rpc_meta_unique_key Int64, - message_index Int32 CODEC(DoubleDelta, ZSTD(1)), - control_index Int32 CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - message_id String CODEC(ZSTD(1)), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, control_index, message_index, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_rpc_meta_control_ihave_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "I have" control messages from the peer.', -COMMENT COLUMN unique_key 'Unique identifier for each "I have" control record', -COMMENT COLUMN updated_date_time 'Timestamp when the "I have" control record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the "I have" control event', -COMMENT COLUMN control_index 'Position in the RPC meta control IWANT array', -COMMENT COLUMN message_index 'Position in the RPC meta control IWANT message_ids array', -COMMENT COLUMN rpc_meta_unique_key 'Unique key associated with the "I have" control metadata', -COMMENT COLUMN topic_layer 'Layer of the topic', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding of the topic', -COMMENT COLUMN message_id 'Identifier of the message associated with the "I have" control', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer involved in the I have control', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_rpc_meta_control_ihave ON CLUSTER '{cluster}' AS libp2p_rpc_meta_control_ihave_local -ENGINE = Distributed('{cluster}', default, libp2p_rpc_meta_control_ihave_local, unique_key); - -CREATE TABLE libp2p_rpc_meta_control_iwant_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - control_index Int32 CODEC(DoubleDelta, ZSTD(1)), - message_index Int32 CODEC(DoubleDelta, ZSTD(1)), - rpc_meta_unique_key Int64, - message_id String CODEC(ZSTD(1)), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, control_index, message_index, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_rpc_meta_control_iwant_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "I want" control messages from the peer.', -COMMENT COLUMN unique_key 'Unique identifier for each "I want" control record', -COMMENT COLUMN updated_date_time 'Timestamp when the "I want" control record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the "I want" control event', -COMMENT COLUMN message_index 'Position in the RPC meta control IWANT message_ids array', -COMMENT COLUMN control_index 'Position in the RPC meta control IWANT array', -COMMENT COLUMN rpc_meta_unique_key 'Unique key associated with the "I want" control metadata', -COMMENT COLUMN message_id 'Identifier of the message associated with the "I want" control', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer involved in the I want control', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_rpc_meta_control_iwant ON CLUSTER '{cluster}' AS libp2p_rpc_meta_control_iwant_local -ENGINE = Distributed('{cluster}', default, libp2p_rpc_meta_control_iwant_local, unique_key); - -CREATE TABLE libp2p_rpc_meta_control_graft_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - control_index Int32 CODEC(DoubleDelta, ZSTD(1)), - rpc_meta_unique_key Int64, - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, control_index, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_rpc_meta_control_graft_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "Graft" control messages from the peer.', -COMMENT COLUMN unique_key 'Unique identifier for each "Graft" control record', -COMMENT COLUMN updated_date_time 'Timestamp when the "Graft" control record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the "Graft" control event', -COMMENT COLUMN control_index 'Position in the RPC meta control GRAFT array', -COMMENT COLUMN rpc_meta_unique_key 'Unique key associated with the "Graft" control metadata', -COMMENT COLUMN topic_layer 'Layer of the topic', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding of the topic', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer involved in the Graft control', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_rpc_meta_control_graft ON CLUSTER '{cluster}' AS libp2p_rpc_meta_control_graft_local -ENGINE = Distributed('{cluster}', default, libp2p_rpc_meta_control_graft_local, unique_key); - -CREATE TABLE libp2p_rpc_meta_control_prune_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - control_index Int32 CODEC(DoubleDelta, ZSTD(1)), - rpc_meta_unique_key Int64, - peer_id_index Int32 CODEC(DoubleDelta, ZSTD(1)), - peer_id_unique_key Int64, - graft_peer_id_unique_key Int64, - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, control_index, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_rpc_meta_control_prune_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "Prune" control messages from the peer.', -COMMENT COLUMN unique_key 'Unique identifier for each "Prune" control record', -COMMENT COLUMN updated_date_time 'Timestamp when the "Prune" control record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the "Prune" control event', -COMMENT COLUMN control_index 'Position in the RPC meta control PRUNE array', -COMMENT COLUMN rpc_meta_unique_key 'Unique key associated with the "Prune" control metadata', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer involved in the Prune control', -COMMENT COLUMN graft_peer_id_unique_key 'Unique key associated with the identifier of the graft peer involved in the Prune control', -COMMENT COLUMN topic_layer 'Layer of the topic', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding of the topic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_rpc_meta_control_prune ON CLUSTER '{cluster}' AS libp2p_rpc_meta_control_prune_local -ENGINE = Distributed('{cluster}', default, libp2p_rpc_meta_control_prune_local, unique_key); - --- Creating local and distributed tables for libp2p_recv_rpc --- unique_key = seahash(.event.id) --- peer_id_unique_key = seahash(data.meta.peer_id + .meta_network_name) --- --- Control messages (prune, iwant, idontwant, etc) are split off from this message --- and stored in separate tables: --- libp2p_rpc_meta_control_prune_local --- libp2p_rpc_meta_control_iwant_local --- libp2p_rpc_meta_control_idontwant_local --- etc. --- --- These control tables reference: --- rpc_meta_unique_key = libp2p_*_rpc_local.unique_key --- peer_id_unique_key = libp2p_*_rpc_local.peer_id_unique_key -CREATE TABLE libp2p_recv_rpc_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_recv_rpc_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC messages received by the peer.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer sender', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_recv_rpc ON CLUSTER '{cluster}' AS libp2p_recv_rpc_local -ENGINE = Distributed('{cluster}', default, libp2p_recv_rpc_local, unique_key); - --- Creating local and distributed tables for libp2p_send_rpc --- unique_key = seahash(.event.id) --- peer_id_unique_key = seahash(data.meta.peer_id + .meta_network_name) --- --- Control messages (prune, iwant, idontwant, etc) are split off from this message --- and stored in separate tables: --- libp2p_rpc_meta_control_prune_local --- libp2p_rpc_meta_control_iwant_local --- libp2p_rpc_meta_control_idontwant_local --- etc. --- --- These control tables reference: --- rpc_meta_unique_key = libp2p_*_rpc_local.unique_key --- peer_id_unique_key = libp2p_*_rpc_local.peer_id_unique_key -CREATE TABLE libp2p_send_rpc_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_send_rpc_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC messages sent by the peer.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer receiver', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_send_rpc ON CLUSTER '{cluster}' AS libp2p_send_rpc_local -ENGINE = Distributed('{cluster}', default, libp2p_send_rpc_local, unique_key); - --- Creating local and distributed tables for libp2p_join -CREATE TABLE libp2p_join_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - peer_id_unique_key Int64, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_join_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the JOIN events from the libp2p client.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN topic_layer 'Layer of the topic', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding of the topic', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer that joined the topic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_join ON CLUSTER '{cluster}' AS libp2p_join_local -ENGINE = Distributed('{cluster}', default, libp2p_join_local, unique_key); - --- Creating local and distributed tables for libp2p_connected -CREATE TABLE libp2p_connected_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - remote_peer_id_unique_key Int64, - remote_protocol LowCardinality(String), - remote_transport_protocol LowCardinality(String), - remote_port UInt16 CODEC(ZSTD(1)), - remote_ip Nullable(IPv6) CODEC(ZSTD(1)), - remote_geo_city LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_country LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - remote_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - remote_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - remote_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - remote_agent_implementation LowCardinality(String), - remote_agent_version LowCardinality(String), - remote_agent_version_major LowCardinality(String), - remote_agent_version_minor LowCardinality(String), - remote_agent_version_patch LowCardinality(String), - remote_agent_platform LowCardinality(String), - direction LowCardinality(String), - opened DateTime CODEC(DoubleDelta, ZSTD(1)), - transient Bool, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_connected_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the CONNECTED events from the libp2p client.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN remote_peer_id_unique_key 'Unique key associated with the identifier of the remote peer', -COMMENT COLUMN remote_protocol 'Protocol of the remote peer', -COMMENT COLUMN remote_transport_protocol 'Transport protocol of the remote peer', -COMMENT COLUMN remote_port 'Port of the remote peer', -COMMENT COLUMN remote_ip 'IP address of the remote peer that generated the event', -COMMENT COLUMN remote_geo_city 'City of the remote peer that generated the event', -COMMENT COLUMN remote_geo_country 'Country of the remote peer that generated the event', -COMMENT COLUMN remote_geo_country_code 'Country code of the remote peer that generated the event', -COMMENT COLUMN remote_geo_continent_code 'Continent code of the remote peer that generated the event', -COMMENT COLUMN remote_geo_longitude 'Longitude of the remote peer that generated the event', -COMMENT COLUMN remote_geo_latitude 'Latitude of the remote peer that generated the event', -COMMENT COLUMN remote_geo_autonomous_system_number 'Autonomous system number of the remote peer that generated the event', -COMMENT COLUMN remote_geo_autonomous_system_organization 'Autonomous system organization of the remote peer that generated the event', -COMMENT COLUMN remote_agent_implementation 'Implementation of the remote peer', -COMMENT COLUMN remote_agent_version 'Version of the remote peer', -COMMENT COLUMN remote_agent_version_major 'Major version of the remote peer', -COMMENT COLUMN remote_agent_version_minor 'Minor version of the remote peer', -COMMENT COLUMN remote_agent_version_patch 'Patch version of the remote peer', -COMMENT COLUMN remote_agent_platform 'Platform of the remote peer', -COMMENT COLUMN direction 'Connection direction', -COMMENT COLUMN opened 'Timestamp when the connection was opened', -COMMENT COLUMN transient 'Whether the connection is transient', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_connected ON CLUSTER '{cluster}' AS libp2p_connected_local -ENGINE = Distributed('{cluster}', default, libp2p_connected_local, unique_key); - --- Creating local and distributed tables for libp2p_disconnected -CREATE TABLE libp2p_disconnected_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - remote_peer_id_unique_key Int64, - remote_protocol LowCardinality(String), - remote_transport_protocol LowCardinality(String), - remote_port UInt16 CODEC(ZSTD(1)), - remote_ip Nullable(IPv6) CODEC(ZSTD(1)), - remote_geo_city LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_country LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - remote_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - remote_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - remote_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - remote_agent_implementation LowCardinality(String), - remote_agent_version LowCardinality(String), - remote_agent_version_major LowCardinality(String), - remote_agent_version_minor LowCardinality(String), - remote_agent_version_patch LowCardinality(String), - remote_agent_platform LowCardinality(String), - direction LowCardinality(String), - opened DateTime CODEC(DoubleDelta, ZSTD(1)), - transient Bool, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_disconnected_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the DISCONNECTED events from the libp2p client.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN remote_peer_id_unique_key 'Unique key associated with the identifier of the remote peer', -COMMENT COLUMN remote_protocol 'Protocol of the remote peer', -COMMENT COLUMN remote_transport_protocol 'Transport protocol of the remote peer', -COMMENT COLUMN remote_port 'Port of the remote peer', -COMMENT COLUMN remote_ip 'IP address of the remote peer that generated the event', -COMMENT COLUMN remote_geo_city 'City of the remote peer that generated the event', -COMMENT COLUMN remote_geo_country 'Country of the remote peer that generated the event', -COMMENT COLUMN remote_geo_country_code 'Country code of the remote peer that generated the event', -COMMENT COLUMN remote_geo_continent_code 'Continent code of the remote peer that generated the event', -COMMENT COLUMN remote_geo_longitude 'Longitude of the remote peer that generated the event', -COMMENT COLUMN remote_geo_latitude 'Latitude of the remote peer that generated the event', -COMMENT COLUMN remote_geo_autonomous_system_number 'Autonomous system number of the remote peer that generated the event', -COMMENT COLUMN remote_geo_autonomous_system_organization 'Autonomous system organization of the remote peer that generated the event', -COMMENT COLUMN remote_agent_implementation 'Implementation of the remote peer', -COMMENT COLUMN remote_agent_version 'Version of the remote peer', -COMMENT COLUMN remote_agent_version_major 'Major version of the remote peer', -COMMENT COLUMN remote_agent_version_minor 'Minor version of the remote peer', -COMMENT COLUMN remote_agent_version_patch 'Patch version of the remote peer', -COMMENT COLUMN remote_agent_platform 'Platform of the remote peer', -COMMENT COLUMN direction 'Connection direction', -COMMENT COLUMN opened 'Timestamp when the connection was opened', -COMMENT COLUMN transient 'Whether the connection is transient', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_disconnected ON CLUSTER '{cluster}' AS libp2p_disconnected_local -ENGINE = Distributed('{cluster}', default, libp2p_disconnected_local, unique_key); diff --git a/deploy/migrations/clickhouse/033_libp2p_trace_metadata_status.down.sql b/deploy/migrations/clickhouse/033_libp2p_trace_metadata_status.down.sql deleted file mode 100644 index a6546b8a..00000000 --- a/deploy/migrations/clickhouse/033_libp2p_trace_metadata_status.down.sql +++ /dev/null @@ -1,5 +0,0 @@ -DROP TABLE IF EXISTS libp2p_handle_status ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS libp2p_handle_status_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS libp2p_handle_metadata ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS libp2p_handle_metadata_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/033_libp2p_trace_metadata_status.up.sql b/deploy/migrations/clickhouse/033_libp2p_trace_metadata_status.up.sql deleted file mode 100644 index 1ec29cac..00000000 --- a/deploy/migrations/clickhouse/033_libp2p_trace_metadata_status.up.sql +++ /dev/null @@ -1,142 +0,0 @@ --- Creating local and distributed tables for libp2p_handle_status -CREATE TABLE libp2p_handle_status_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - peer_id_unique_key Int64, - error Nullable(String) CODEC(ZSTD(1)), - protocol LowCardinality(String), - request_finalized_epoch Nullable(UInt32) CODEC(DoubleDelta, ZSTD(1)), - request_finalized_root Nullable(String), - request_fork_digest LowCardinality(String), - request_head_root Nullable(FixedString(66)) CODEC(ZSTD(1)), - request_head_slot Nullable(UInt32) CODEC(ZSTD(1)), - response_finalized_epoch Nullable(UInt32) CODEC(DoubleDelta, ZSTD(1)), - response_finalized_root Nullable(FixedString(66)) CODEC(ZSTD(1)), - response_fork_digest LowCardinality(String), - response_head_root Nullable(FixedString(66)) CODEC(ZSTD(1)), - response_head_slot Nullable(UInt32) CODEC(DoubleDelta, ZSTD(1)), - latency_milliseconds Decimal(10,3) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_handle_status_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the status handling events for libp2p peers.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN error 'Error message if the status handling failed', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer', -COMMENT COLUMN protocol 'The protocol of the status handling event', -COMMENT COLUMN request_finalized_epoch 'Requested finalized epoch', -COMMENT COLUMN request_finalized_root 'Requested finalized root', -COMMENT COLUMN request_fork_digest 'Requested fork digest', -COMMENT COLUMN request_head_root 'Requested head root', -COMMENT COLUMN request_head_slot 'Requested head slot', -COMMENT COLUMN response_finalized_epoch 'Response finalized epoch', -COMMENT COLUMN response_finalized_root 'Response finalized root', -COMMENT COLUMN response_fork_digest 'Response fork digest', -COMMENT COLUMN response_head_root 'Response head root', -COMMENT COLUMN response_head_slot 'Response head slot', -COMMENT COLUMN latency_milliseconds 'How long it took to handle the status request in milliseconds', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_handle_status ON CLUSTER '{cluster}' AS libp2p_handle_status_local -ENGINE = Distributed('{cluster}', default, libp2p_handle_status_local, unique_key); - -CREATE TABLE libp2p_handle_metadata_local ON CLUSTER '{cluster}' -( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - peer_id_unique_key Int64, - error Nullable(String) CODEC(ZSTD(1)), - protocol LowCardinality(String), - attnets String CODEC(ZSTD(1)), - seq_number UInt64 CODEC(DoubleDelta, ZSTD(1)), - syncnets String CODEC(ZSTD(1)), - latency_milliseconds Decimal(10,3) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_handle_metadata_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the metadata handling events for libp2p peers.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN event_date_time 'Timestamp of the event', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN error 'Error message if the metadata handling failed', -COMMENT COLUMN protocol 'The protocol of the metadata handling event', -COMMENT COLUMN attnets 'Attestation subnets the peer is subscribed to', -COMMENT COLUMN seq_number 'Sequence number of the metadata', -COMMENT COLUMN syncnets 'Sync subnets the peer is subscribed to', -COMMENT COLUMN latency_milliseconds 'How long it took to handle the metadata request in milliseconds', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer involved in the RPC', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_handle_metadata ON CLUSTER '{cluster}' AS libp2p_handle_metadata_local -ENGINE = Distributed('{cluster}', default, libp2p_handle_metadata_local, unique_key); diff --git a/deploy/migrations/clickhouse/034_libp2p_gossipsub_beacon_block.down.sql b/deploy/migrations/clickhouse/034_libp2p_gossipsub_beacon_block.down.sql deleted file mode 100644 index 728ee79f..00000000 --- a/deploy/migrations/clickhouse/034_libp2p_gossipsub_beacon_block.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/034_libp2p_gossipsub_beacon_block.up.sql b/deploy/migrations/clickhouse/034_libp2p_gossipsub_beacon_block.up.sql deleted file mode 100644 index 8293d9d5..00000000 --- a/deploy/migrations/clickhouse/034_libp2p_gossipsub_beacon_block.up.sql +++ /dev/null @@ -1,84 +0,0 @@ -CREATE TABLE libp2p_gossipsub_beacon_block_local on cluster '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - wallclock_slot UInt32 Codec(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - wallclock_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - wallclock_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - block FixedString(66) Codec(ZSTD(1)), - proposer_index UInt32 CODEC(ZSTD(1)), - peer_id_unique_key Int64, - message_id String CODEC(ZSTD(1)), - message_size UInt32 Codec(ZSTD(1)), - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub beacon block data.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event with millisecond precision', -COMMENT COLUMN slot 'Slot number associated with the event', -COMMENT COLUMN slot_start_date_time 'Start date and time of the slot', -COMMENT COLUMN epoch 'Epoch number associated with the event', -COMMENT COLUMN epoch_start_date_time 'Start date and time of the epoch', -COMMENT COLUMN wallclock_slot 'Slot number of the wall clock when the event was received', -COMMENT COLUMN wallclock_slot_start_date_time 'Start date and time of the wall clock slot when the event was received', -COMMENT COLUMN wallclock_epoch 'Epoch number of the wall clock when the event was received', -COMMENT COLUMN wallclock_epoch_start_date_time 'Start date and time of the wall clock epoch when the event was received', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slot start time for propagation', -COMMENT COLUMN proposer_index 'The proposer index of the beacon block', -COMMENT COLUMN block 'The beacon block root hash', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer', -COMMENT COLUMN message_id 'Identifier of the message', -COMMENT COLUMN message_size 'Size of the message in bytes', -COMMENT COLUMN topic_layer 'Layer of the topic in the gossipsub protocol', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding used for the topic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Network ID associated with the client', -COMMENT COLUMN meta_network_name 'Name of the network associated with the client'; - -CREATE TABLE libp2p_gossipsub_beacon_block on cluster '{cluster}' AS libp2p_gossipsub_beacon_block_local -ENGINE = Distributed('{cluster}', default, libp2p_gossipsub_beacon_block_local, unique_key); diff --git a/deploy/migrations/clickhouse/035_libp2p_gossipsub_beacon_attestation.down.sql b/deploy/migrations/clickhouse/035_libp2p_gossipsub_beacon_attestation.down.sql deleted file mode 100644 index 63f8a094..00000000 --- a/deploy/migrations/clickhouse/035_libp2p_gossipsub_beacon_attestation.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/035_libp2p_gossipsub_beacon_attestation.up.sql b/deploy/migrations/clickhouse/035_libp2p_gossipsub_beacon_attestation.up.sql deleted file mode 100644 index 4bb1dd2d..00000000 --- a/deploy/migrations/clickhouse/035_libp2p_gossipsub_beacon_attestation.up.sql +++ /dev/null @@ -1,102 +0,0 @@ -CREATE TABLE libp2p_gossipsub_beacon_attestation_local on cluster '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - committee_index LowCardinality(String), - attesting_validator_index Nullable(UInt32) Codec(ZSTD(1)), - attesting_validator_committee_index LowCardinality(String), - wallclock_slot UInt32 Codec(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - wallclock_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - wallclock_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - peer_id_unique_key Int64, - message_id String CODEC(ZSTD(1)), - message_size UInt32 Codec(ZSTD(1)), - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - aggregation_bits String Codec(ZSTD(1)), - beacon_block_root FixedString(66) Codec(ZSTD(1)), - source_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - source_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - source_root FixedString(66) Codec(ZSTD(1)), - target_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - target_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - target_root FixedString(66) Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub beacon attestation data.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event with millisecond precision', -COMMENT COLUMN slot 'Slot number associated with the event', -COMMENT COLUMN slot_start_date_time 'Start date and time of the slot', -COMMENT COLUMN wallclock_slot 'Slot number of the wall clock when the event was received', -COMMENT COLUMN wallclock_slot_start_date_time 'Start date and time of the wall clock slot when the event was received', -COMMENT COLUMN wallclock_epoch 'Epoch number of the wall clock when the event was received', -COMMENT COLUMN wallclock_epoch_start_date_time 'Start date and time of the wall clock epoch when the event was received', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slot start time for propagation', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer', -COMMENT COLUMN message_id 'Identifier of the message', -COMMENT COLUMN message_size 'Size of the message in bytes', -COMMENT COLUMN topic_layer 'Layer of the topic in the gossipsub protocol', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding used for the topic', -COMMENT COLUMN aggregation_bits 'The aggregation bits of the event in the attestation', -COMMENT COLUMN beacon_block_root 'The beacon block root hash in the attestation', -COMMENT COLUMN epoch 'The epoch number in the attestation', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN source_epoch 'The source epoch number in the attestation', -COMMENT COLUMN source_epoch_start_date_time 'The wall clock time when the source epoch started', -COMMENT COLUMN source_root 'The source beacon block root hash in the attestation', -COMMENT COLUMN target_epoch 'The target epoch number in the attestation', -COMMENT COLUMN target_epoch_start_date_time 'The wall clock time when the target epoch started', -COMMENT COLUMN target_root 'The target beacon block root hash in the attestation', -COMMENT COLUMN committee_index 'The committee index in the attestation', -COMMENT COLUMN attesting_validator_index 'The index of the validator attesting to the event', -COMMENT COLUMN attesting_validator_committee_index 'The committee index of the attesting validator', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Network ID associated with the client', -COMMENT COLUMN meta_network_name 'Name of the network associated with the client'; - -CREATE TABLE libp2p_gossipsub_beacon_attestation on cluster '{cluster}' AS libp2p_gossipsub_beacon_attestation_local -ENGINE = Distributed('{cluster}', default, libp2p_gossipsub_beacon_attestation_local, unique_key); diff --git a/deploy/migrations/clickhouse/036_replacing_merge_trees.down.sql b/deploy/migrations/clickhouse/036_replacing_merge_trees.down.sql deleted file mode 100644 index af82275a..00000000 --- a/deploy/migrations/clickhouse/036_replacing_merge_trees.down.sql +++ /dev/null @@ -1,1104 +0,0 @@ --- beacon_api_eth_v1_beacon_committee -CREATE TABLE tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - committee_index LowCardinality(String), - validators Array(UInt32) CODEC(ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API committee payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN committee_index 'The committee index in the beacon API committee payload', -COMMENT COLUMN validators 'The validator indices in the beacon API committee payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API committee payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_beacon_committee_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_beacon_committee_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_beacon_committee -SELECT - toInt64(cityHash64(toString(slot) || committee_index || toString(validators) || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_beacon_committee_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_beacon_committee_local AND tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_beacon_committee_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_beacon_committee_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_blob_sidecar -CREATE TABLE tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 CODEC(ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - blob_index UInt64 CODEC(ZSTD(1)), - kzg_commitment FixedString(98) CODEC(ZSTD(1)), - versioned_hash FixedString(66) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN blob_index 'The index of blob sidecar in the beacon API event stream payload', -COMMENT COLUMN kzg_commitment 'The KZG commitment in the beacon API event stream payload', -COMMENT COLUMN versioned_hash 'The versioned hash in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_blob_sidecar_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_blob_sidecar_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_blob_sidecar -SELECT - toInt64(cityHash64(toString(slot) || toString(blob_index) || block_root || kzg_commitment || versioned_hash || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_blob_sidecar_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_blob_sidecar_local AND tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_blob_sidecar_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_blob_sidecar_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_block -CREATE TABLE tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 CODEC(ZSTD(1)), - block FixedString(66) CODEC(ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN block 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'If the attached beacon node is running in execution optimistic mode', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_block_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_block_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_block -SELECT - toInt64(cityHash64(toString(slot) || block || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_block_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_block_local AND tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_block_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_block_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_chain_reorg -CREATE TABLE tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - slot UInt32, - slot_start_date_time DateTime, - propagation_slot_start_diff UInt32, - depth UInt16, - old_head_block FixedString(66), - new_head_block FixedString(66), - old_head_state FixedString(66), - new_head_state FixedString(66), - epoch UInt32, - epoch_start_date_time DateTime, - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'The slot number of the chain reorg event in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the reorg slot started', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slots between when the reorg occurred and when the sentry received the event', -COMMENT COLUMN depth 'The depth of the chain reorg in the beacon API event stream payload', -COMMENT COLUMN old_head_block 'The old head block root hash in the beacon API event stream payload', -COMMENT COLUMN new_head_block 'The new head block root hash in the beacon API event stream payload', -COMMENT COLUMN old_head_state 'The old head state root hash in the beacon API event stream payload', -COMMENT COLUMN new_head_state 'The new head state root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'Whether the execution of the epoch was optimistic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_chain_reorg_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_chain_reorg_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_chain_reorg -SELECT - toInt64(cityHash64(toString(slot) || toString(depth) || old_head_block || new_head_block || old_head_state || new_head_state || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_chain_reorg_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_chain_reorg_local AND tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_chain_reorg_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_chain_reorg_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_contribution_and_proof -CREATE TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - aggregator_index UInt32, - contribution_slot UInt32, - contribution_slot_start_date_time DateTime, - contribution_propagation_slot_start_diff UInt32, - contribution_beacon_block_root FixedString(66), - contribution_subcommittee_index LowCardinality(String), - contribution_aggregation_bits String, - contribution_signature String, - contribution_epoch UInt32, - contribution_epoch_start_date_time DateTime, - selection_proof String, - signature String, - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(contribution_slot_start_date_time) -ORDER BY (contribution_slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN aggregator_index 'The validator index of the aggregator in the beacon API event stream payload', -COMMENT COLUMN contribution_slot 'The slot number of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_slot_start_date_time 'The wall clock time when the contribution slot started', -COMMENT COLUMN contribution_propagation_slot_start_diff 'Difference in slots between when the contribution occurred and when the sentry received the event', -COMMENT COLUMN contribution_beacon_block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN contribution_subcommittee_index 'The subcommittee index of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_aggregation_bits 'The aggregation bits of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_signature 'The signature of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_epoch 'The epoch number of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_epoch_start_date_time 'The wall clock time when the contribution epoch started', -COMMENT COLUMN selection_proof 'The selection proof in the beacon API event stream payload', -COMMENT COLUMN signature 'The signature in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_contribution_and_proof_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_contribution_and_proof_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_contribution_and_proof -SELECT - toInt64(cityHash64(toString(aggregator_index) || toString(contribution_slot) || contribution_beacon_block_root || contribution_subcommittee_index || contribution_aggregation_bits || contribution_signature || selection_proof || signature || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_contribution_and_proof_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_contribution_and_proof_local AND tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_contribution_and_proof_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_contribution_and_proof_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_finalized_checkpoint -CREATE TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - block FixedString(66) CODEC(ZSTD(1)), - state FixedString(66) CODEC(ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN block 'The finalized block root hash in the beacon API event stream payload', -COMMENT COLUMN state 'The finalized state root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'Whether the execution of the epoch was optimistic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_finalized_checkpoint_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_finalized_checkpoint_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_finalized_checkpoint -SELECT - toInt64(cityHash64(toString(epoch) || block || state || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_finalized_checkpoint_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_finalized_checkpoint_local AND tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_finalized_checkpoint_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_finalized_checkpoint_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_head -CREATE TABLE tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - slot UInt32, - slot_start_date_time DateTime, - propagation_slot_start_diff UInt32, - block FixedString(66), - epoch UInt32, - epoch_start_date_time DateTime, - epoch_transition Bool, - execution_optimistic Bool, - previous_duty_dependent_root FixedString(66), - current_duty_dependent_root FixedString(66), - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN block 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN epoch_transition 'If the event is an epoch transition', -COMMENT COLUMN execution_optimistic 'If the attached beacon node is running in execution optimistic mode', -COMMENT COLUMN previous_duty_dependent_root 'The previous duty dependent root in the beacon API event stream payload', -COMMENT COLUMN current_duty_dependent_root 'The current duty dependent root in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_head_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_head_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_head -SELECT - toInt64(cityHash64(toString(slot) || block || previous_duty_dependent_root || current_duty_dependent_root || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_head_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_head_local AND tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_head_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_head_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_voluntary_exit -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - epoch UInt32, - epoch_start_date_time DateTime, - validator_index UInt32, - signature String, - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN validator_index 'The index of the validator making the voluntary exit', -COMMENT COLUMN signature 'The signature of the voluntary exit in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_voluntary_exit_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_voluntary_exit_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_voluntary_exit -SELECT - toInt64(cityHash64(toString(epoch) || validator_index || signature || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_voluntary_exit_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_voluntary_exit_local AND tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_voluntary_exit_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_validator_attestation_data -CREATE TABLE tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - slot UInt32, - slot_start_date_time DateTime, - committee_index LowCardinality(String), - beacon_block_root FixedString(66), - epoch UInt32, - epoch_start_date_time DateTime, - source_epoch UInt32, - source_epoch_start_date_time DateTime, - source_root FixedString(66), - target_epoch UInt32, - target_epoch_start_date_time DateTime, - target_root FixedString(66), - request_date_time DateTime, - request_duration UInt32, - request_slot_start_diff UInt32, - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API validator attestation data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API validator attestation data payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN committee_index 'The committee index in the beacon API validator attestation data payload', -COMMENT COLUMN beacon_block_root 'The beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN source_epoch 'The source epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN source_epoch_start_date_time 'The wall clock time when the source epoch started', -COMMENT COLUMN source_root 'The source beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN target_epoch 'The target epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN target_epoch_start_date_time 'The wall clock time when the target epoch started', -COMMENT COLUMN target_root 'The target beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN request_date_time 'When the request was sent to the beacon node', -COMMENT COLUMN request_duration 'The request duration in milliseconds', -COMMENT COLUMN request_slot_start_diff 'The difference between the request_date_time and the slot_start_date_time', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_validator_attestation_data_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_validator_attestation_data_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_validator_attestation_data -SELECT - toInt64(cityHash64(toString(slot) || committee_index || beacon_block_root || toString(source_epoch) || source_root || toString(target_epoch) || target_root || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_validator_attestation_data_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_validator_attestation_data_local AND tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_validator_attestation_data_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_validator_attestation_data_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v2_beacon_block -CREATE TABLE tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - slot UInt32, - slot_start_date_time DateTime, - epoch UInt32, - epoch_start_date_time DateTime, - block_root FixedString(66), - block_version LowCardinality(String), - block_total_bytes Nullable(UInt32), - block_total_bytes_compressed Nullable(UInt32), - parent_root FixedString(66), - state_root FixedString(66), - proposer_index UInt32, - eth1_data_block_hash FixedString(66), - eth1_data_deposit_root FixedString(66), - execution_payload_block_hash FixedString(66), - execution_payload_block_number UInt32, - execution_payload_fee_recipient String, - execution_payload_state_root FixedString(66), - execution_payload_parent_hash FixedString(66), - execution_payload_transactions_count Nullable(UInt32), - execution_payload_transactions_total_bytes Nullable(UInt32), - execution_payload_transactions_total_bytes_compressed Nullable(UInt32), - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_execution_fork_id_hash LowCardinality(String), - meta_execution_fork_id_next LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the reorg slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN block_total_bytes 'The total bytes of the beacon block payload', -COMMENT COLUMN block_total_bytes_compressed 'The total bytes of the beacon block payload when compressed using snappy', -COMMENT COLUMN parent_root 'The root hash of the parent beacon block', -COMMENT COLUMN state_root 'The root hash of the beacon state at this block', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN eth1_data_block_hash 'The block hash of the associated execution block', -COMMENT COLUMN eth1_data_deposit_root 'The root of the deposit tree in the associated execution block', -COMMENT COLUMN execution_payload_block_hash 'The block hash of the execution payload', -COMMENT COLUMN execution_payload_block_number 'The block number of the execution payload', -COMMENT COLUMN execution_payload_fee_recipient 'The recipient of the fee for this execution payload', -COMMENT COLUMN execution_payload_state_root 'The state root of the execution payload', -COMMENT COLUMN execution_payload_parent_hash 'The parent hash of the execution payload', -COMMENT COLUMN execution_payload_transactions_count 'The transaction count of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes 'The transaction total bytes of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes_compressed 'The transaction total bytes of the execution payload when compressed using snappy', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_execution_fork_id_hash 'The hash of the fork ID of the current Ethereum network', -COMMENT COLUMN meta_execution_fork_id_next 'The fork ID of the next planned Ethereum network upgrade', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v2_beacon_block_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v2_beacon_block_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v2_beacon_block -SELECT - toInt64(cityHash64(toString(slot)|| block_root || block_version || parent_root || state_root || toString(proposer_index) || eth1_data_block_hash || execution_payload_block_hash || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v2_beacon_block_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' SYNC; - -ALTER TABLE tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' - DROP COLUMN meta_execution_fork_id_hash, - DROP COLUMN meta_execution_fork_id_next; - -EXCHANGE TABLES default.beacon_api_eth_v2_beacon_block_local AND tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v2_beacon_block_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' SYNC; - --- mempool_transaction -CREATE TABLE tmp.mempool_transaction_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - hash FixedString(66) CODEC(ZSTD(1)), - from FixedString(42) CODEC(ZSTD(1)), - to Nullable(FixedString(42)) CODEC(ZSTD(1)), - nonce UInt64 CODEC(ZSTD(1)), - gas_price UInt128 CODEC(ZSTD(1)), - gas UInt64 CODEC(ZSTD(1)), - gas_tip_cap Nullable(UInt128), - gas_fee_cap Nullable(UInt128), - value UInt128 CODEC(ZSTD(1)), - type Nullable(UInt8), - size UInt32 CODEC(ZSTD(1)), - call_data_size UInt32 CODEC(ZSTD(1)), - blob_gas Nullable(UInt64), - blob_gas_fee_cap Nullable(UInt128), - blob_hashes Array(String), - blob_sidecars_size Nullable(UInt32), - blob_sidecars_empty_size Nullable(UInt32), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_execution_fork_id_hash LowCardinality(String), - meta_execution_fork_id_next LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.mempool_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Each row represents a transaction that was seen in the mempool by a sentry client. Sentries can report the same transaction multiple times if it has been long enough since the last report.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'The time when the sentry saw the transaction in the mempool', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN gas_tip_cap 'The priority fee (tip) the user has set for the transaction', -COMMENT COLUMN gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN type 'The type of the transaction', -COMMENT COLUMN size 'The size of the transaction data in bytes', -COMMENT COLUMN call_data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN blob_gas 'The maximum gas provided for the blob transaction execution', -COMMENT COLUMN blob_gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_hashes 'The hashes of the blob commitments for blob transactions', -COMMENT COLUMN blob_sidecars_size 'The total size of the sidecars for blob transactions in bytes', -COMMENT COLUMN blob_sidecars_empty_size 'The total empty size of the sidecars for blob transactions in bytes', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_execution_fork_id_hash 'The hash of the fork ID of the current Ethereum network', -COMMENT COLUMN meta_execution_fork_id_next 'The fork ID of the next planned Ethereum network upgrade', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.mempool_transaction ON CLUSTER '{cluster}' AS tmp.mempool_transaction_local -ENGINE = Distributed('{cluster}', tmp, mempool_transaction_local, unique_key); - -INSERT INTO tmp.mempool_transaction -SELECT - toInt64(cityHash64(hash || from || to || toString(nonce) || toString(type) || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.mempool_transaction_local; - -DROP TABLE IF EXISTS default.mempool_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.mempool_transaction_local AND tmp.mempool_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.mempool_transaction ON CLUSTER '{cluster}' AS default.mempool_transaction_local -ENGINE = Distributed('{cluster}', default, mempool_transaction_local, unique_key); - -DROP TABLE IF EXISTS tmp.mempool_transaction ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.mempool_transaction_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/036_replacing_merge_trees.up.sql b/deploy/migrations/clickhouse/036_replacing_merge_trees.up.sql deleted file mode 100644 index af82275a..00000000 --- a/deploy/migrations/clickhouse/036_replacing_merge_trees.up.sql +++ /dev/null @@ -1,1104 +0,0 @@ --- beacon_api_eth_v1_beacon_committee -CREATE TABLE tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - committee_index LowCardinality(String), - validators Array(UInt32) CODEC(ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API committee payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN committee_index 'The committee index in the beacon API committee payload', -COMMENT COLUMN validators 'The validator indices in the beacon API committee payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API committee payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_beacon_committee_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_beacon_committee_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_beacon_committee -SELECT - toInt64(cityHash64(toString(slot) || committee_index || toString(validators) || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_beacon_committee_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_beacon_committee_local AND tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_beacon_committee_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_beacon_committee_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_blob_sidecar -CREATE TABLE tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 CODEC(ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - block_root FixedString(66) CODEC(ZSTD(1)), - blob_index UInt64 CODEC(ZSTD(1)), - kzg_commitment FixedString(98) CODEC(ZSTD(1)), - versioned_hash FixedString(66) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN blob_index 'The index of blob sidecar in the beacon API event stream payload', -COMMENT COLUMN kzg_commitment 'The KZG commitment in the beacon API event stream payload', -COMMENT COLUMN versioned_hash 'The versioned hash in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_blob_sidecar_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_blob_sidecar_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_blob_sidecar -SELECT - toInt64(cityHash64(toString(slot) || toString(blob_index) || block_root || kzg_commitment || versioned_hash || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_blob_sidecar_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_blob_sidecar_local AND tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_blob_sidecar_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_blob_sidecar_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_block -CREATE TABLE tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 CODEC(ZSTD(1)), - block FixedString(66) CODEC(ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN block 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'If the attached beacon node is running in execution optimistic mode', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_block_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_block_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_block -SELECT - toInt64(cityHash64(toString(slot) || block || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_block_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_block_local AND tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_block_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_block_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_chain_reorg -CREATE TABLE tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - slot UInt32, - slot_start_date_time DateTime, - propagation_slot_start_diff UInt32, - depth UInt16, - old_head_block FixedString(66), - new_head_block FixedString(66), - old_head_state FixedString(66), - new_head_state FixedString(66), - epoch UInt32, - epoch_start_date_time DateTime, - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'The slot number of the chain reorg event in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the reorg slot started', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slots between when the reorg occurred and when the sentry received the event', -COMMENT COLUMN depth 'The depth of the chain reorg in the beacon API event stream payload', -COMMENT COLUMN old_head_block 'The old head block root hash in the beacon API event stream payload', -COMMENT COLUMN new_head_block 'The new head block root hash in the beacon API event stream payload', -COMMENT COLUMN old_head_state 'The old head state root hash in the beacon API event stream payload', -COMMENT COLUMN new_head_state 'The new head state root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'Whether the execution of the epoch was optimistic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_chain_reorg_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_chain_reorg_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_chain_reorg -SELECT - toInt64(cityHash64(toString(slot) || toString(depth) || old_head_block || new_head_block || old_head_state || new_head_state || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_chain_reorg_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_chain_reorg_local AND tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_chain_reorg_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_chain_reorg_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_contribution_and_proof -CREATE TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - aggregator_index UInt32, - contribution_slot UInt32, - contribution_slot_start_date_time DateTime, - contribution_propagation_slot_start_diff UInt32, - contribution_beacon_block_root FixedString(66), - contribution_subcommittee_index LowCardinality(String), - contribution_aggregation_bits String, - contribution_signature String, - contribution_epoch UInt32, - contribution_epoch_start_date_time DateTime, - selection_proof String, - signature String, - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(contribution_slot_start_date_time) -ORDER BY (contribution_slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN aggregator_index 'The validator index of the aggregator in the beacon API event stream payload', -COMMENT COLUMN contribution_slot 'The slot number of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_slot_start_date_time 'The wall clock time when the contribution slot started', -COMMENT COLUMN contribution_propagation_slot_start_diff 'Difference in slots between when the contribution occurred and when the sentry received the event', -COMMENT COLUMN contribution_beacon_block_root 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN contribution_subcommittee_index 'The subcommittee index of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_aggregation_bits 'The aggregation bits of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_signature 'The signature of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_epoch 'The epoch number of the contribution in the beacon API event stream payload', -COMMENT COLUMN contribution_epoch_start_date_time 'The wall clock time when the contribution epoch started', -COMMENT COLUMN selection_proof 'The selection proof in the beacon API event stream payload', -COMMENT COLUMN signature 'The signature in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_contribution_and_proof_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_contribution_and_proof_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_contribution_and_proof -SELECT - toInt64(cityHash64(toString(aggregator_index) || toString(contribution_slot) || contribution_beacon_block_root || contribution_subcommittee_index || contribution_aggregation_bits || contribution_signature || selection_proof || signature || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_contribution_and_proof_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_contribution_and_proof_local AND tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_contribution_and_proof_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_contribution_and_proof_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_finalized_checkpoint -CREATE TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - block FixedString(66) CODEC(ZSTD(1)), - state FixedString(66) CODEC(ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - execution_optimistic Bool, - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN block 'The finalized block root hash in the beacon API event stream payload', -COMMENT COLUMN state 'The finalized state root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN execution_optimistic 'Whether the execution of the epoch was optimistic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_finalized_checkpoint_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_finalized_checkpoint_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_finalized_checkpoint -SELECT - toInt64(cityHash64(toString(epoch) || block || state || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_finalized_checkpoint_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_finalized_checkpoint_local AND tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_finalized_checkpoint_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_finalized_checkpoint_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_head -CREATE TABLE tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - slot UInt32, - slot_start_date_time DateTime, - propagation_slot_start_diff UInt32, - block FixedString(66), - epoch UInt32, - epoch_start_date_time DateTime, - epoch_transition Bool, - execution_optimistic Bool, - previous_duty_dependent_root FixedString(66), - current_duty_dependent_root FixedString(66), - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API event stream payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time', -COMMENT COLUMN block 'The beacon block root hash in the beacon API event stream payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN epoch_transition 'If the event is an epoch transition', -COMMENT COLUMN execution_optimistic 'If the attached beacon node is running in execution optimistic mode', -COMMENT COLUMN previous_duty_dependent_root 'The previous duty dependent root in the beacon API event stream payload', -COMMENT COLUMN current_duty_dependent_root 'The current duty dependent root in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_head_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_head_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_head -SELECT - toInt64(cityHash64(toString(slot) || block || previous_duty_dependent_root || current_duty_dependent_root || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_head_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_head_local AND tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_head_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_head_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_voluntary_exit -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - epoch UInt32, - epoch_start_date_time DateTime, - validator_index UInt32, - signature String, - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN epoch 'The epoch number in the beacon API event stream payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN validator_index 'The index of the validator making the voluntary exit', -COMMENT COLUMN signature 'The signature of the voluntary exit in the beacon API event stream payload', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_voluntary_exit_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_events_voluntary_exit_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_events_voluntary_exit -SELECT - toInt64(cityHash64(toString(epoch) || validator_index || signature || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_events_voluntary_exit_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_voluntary_exit_local AND tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_voluntary_exit_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_events_voluntary_exit_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_validator_attestation_data -CREATE TABLE tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - slot UInt32, - slot_start_date_time DateTime, - committee_index LowCardinality(String), - beacon_block_root FixedString(66), - epoch UInt32, - epoch_start_date_time DateTime, - source_epoch UInt32, - source_epoch_start_date_time DateTime, - source_root FixedString(66), - target_epoch UInt32, - target_epoch_start_date_time DateTime, - target_root FixedString(66), - request_date_time DateTime, - request_duration UInt32, - request_slot_start_diff UInt32, - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API validator attestation data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon API validator attestation data payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN committee_index 'The committee index in the beacon API validator attestation data payload', -COMMENT COLUMN beacon_block_root 'The beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN epoch 'The epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN source_epoch 'The source epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN source_epoch_start_date_time 'The wall clock time when the source epoch started', -COMMENT COLUMN source_root 'The source beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN target_epoch 'The target epoch number in the beacon API validator attestation data payload', -COMMENT COLUMN target_epoch_start_date_time 'The wall clock time when the target epoch started', -COMMENT COLUMN target_root 'The target beacon block root hash in the beacon API validator attestation data payload', -COMMENT COLUMN request_date_time 'When the request was sent to the beacon node', -COMMENT COLUMN request_duration 'The request duration in milliseconds', -COMMENT COLUMN request_slot_start_diff 'The difference between the request_date_time and the slot_start_date_time', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_validator_attestation_data_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v1_validator_attestation_data_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v1_validator_attestation_data -SELECT - toInt64(cityHash64(toString(slot) || committee_index || beacon_block_root || toString(source_epoch) || source_root || toString(target_epoch) || target_root || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v1_validator_attestation_data_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_validator_attestation_data_local AND tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_validator_attestation_data_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_validator_attestation_data_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v2_beacon_block -CREATE TABLE tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3), - slot UInt32, - slot_start_date_time DateTime, - epoch UInt32, - epoch_start_date_time DateTime, - block_root FixedString(66), - block_version LowCardinality(String), - block_total_bytes Nullable(UInt32), - block_total_bytes_compressed Nullable(UInt32), - parent_root FixedString(66), - state_root FixedString(66), - proposer_index UInt32, - eth1_data_block_hash FixedString(66), - eth1_data_deposit_root FixedString(66), - execution_payload_block_hash FixedString(66), - execution_payload_block_number UInt32, - execution_payload_fee_recipient String, - execution_payload_state_root FixedString(66), - execution_payload_parent_hash FixedString(66), - execution_payload_transactions_count Nullable(UInt32), - execution_payload_transactions_total_bytes Nullable(UInt32), - execution_payload_transactions_total_bytes_compressed Nullable(UInt32), - meta_client_name LowCardinality(String), - meta_client_id String, - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6), - meta_client_geo_city LowCardinality(String), - meta_client_geo_country LowCardinality(String), - meta_client_geo_country_code LowCardinality(String), - meta_client_geo_continent_code LowCardinality(String), - meta_client_geo_longitude Nullable(Float64), - meta_client_geo_latitude Nullable(Float64), - meta_client_geo_autonomous_system_number Nullable(UInt32), - meta_client_geo_autonomous_system_organization Nullable(String), - meta_network_id Int32, - meta_network_name LowCardinality(String), - meta_execution_fork_id_hash LowCardinality(String), - meta_execution_fork_id_next LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'When the sentry fetched the beacon block from a beacon node', -COMMENT COLUMN slot 'The slot number from beacon block payload', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the reorg slot started', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The root hash of the beacon block', -COMMENT COLUMN block_version 'The version of the beacon block', -COMMENT COLUMN block_total_bytes 'The total bytes of the beacon block payload', -COMMENT COLUMN block_total_bytes_compressed 'The total bytes of the beacon block payload when compressed using snappy', -COMMENT COLUMN parent_root 'The root hash of the parent beacon block', -COMMENT COLUMN state_root 'The root hash of the beacon state at this block', -COMMENT COLUMN proposer_index 'The index of the validator that proposed the beacon block', -COMMENT COLUMN eth1_data_block_hash 'The block hash of the associated execution block', -COMMENT COLUMN eth1_data_deposit_root 'The root of the deposit tree in the associated execution block', -COMMENT COLUMN execution_payload_block_hash 'The block hash of the execution payload', -COMMENT COLUMN execution_payload_block_number 'The block number of the execution payload', -COMMENT COLUMN execution_payload_fee_recipient 'The recipient of the fee for this execution payload', -COMMENT COLUMN execution_payload_state_root 'The state root of the execution payload', -COMMENT COLUMN execution_payload_parent_hash 'The parent hash of the execution payload', -COMMENT COLUMN execution_payload_transactions_count 'The transaction count of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes 'The transaction total bytes of the execution payload', -COMMENT COLUMN execution_payload_transactions_total_bytes_compressed 'The transaction total bytes of the execution payload when compressed using snappy', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_execution_fork_id_hash 'The hash of the fork ID of the current Ethereum network', -COMMENT COLUMN meta_execution_fork_id_next 'The fork ID of the next planned Ethereum network upgrade', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v2_beacon_block_local -ENGINE = Distributed('{cluster}', tmp, beacon_api_eth_v2_beacon_block_local, unique_key); - -INSERT INTO tmp.beacon_api_eth_v2_beacon_block -SELECT - toInt64(cityHash64(toString(slot)|| block_root || block_version || parent_root || state_root || toString(proposer_index) || eth1_data_block_hash || execution_payload_block_hash || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.beacon_api_eth_v2_beacon_block_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' SYNC; - -ALTER TABLE tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' - DROP COLUMN meta_execution_fork_id_hash, - DROP COLUMN meta_execution_fork_id_next; - -EXCHANGE TABLES default.beacon_api_eth_v2_beacon_block_local AND tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v2_beacon_block_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v2_beacon_block_local, unique_key); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' SYNC; - --- mempool_transaction -CREATE TABLE tmp.mempool_transaction_local ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - hash FixedString(66) CODEC(ZSTD(1)), - from FixedString(42) CODEC(ZSTD(1)), - to Nullable(FixedString(42)) CODEC(ZSTD(1)), - nonce UInt64 CODEC(ZSTD(1)), - gas_price UInt128 CODEC(ZSTD(1)), - gas UInt64 CODEC(ZSTD(1)), - gas_tip_cap Nullable(UInt128), - gas_fee_cap Nullable(UInt128), - value UInt128 CODEC(ZSTD(1)), - type Nullable(UInt8), - size UInt32 CODEC(ZSTD(1)), - call_data_size UInt32 CODEC(ZSTD(1)), - blob_gas Nullable(UInt64), - blob_gas_fee_cap Nullable(UInt128), - blob_hashes Array(String), - blob_sidecars_size Nullable(UInt32), - blob_sidecars_empty_size Nullable(UInt32), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_execution_fork_id_hash LowCardinality(String), - meta_execution_fork_id_next LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE tmp.mempool_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Each row represents a transaction that was seen in the mempool by a sentry client. Sentries can report the same transaction multiple times if it has been long enough since the last report.', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'The time when the sentry saw the transaction in the mempool', -COMMENT COLUMN hash 'The hash of the transaction', -COMMENT COLUMN from 'The address of the account that sent the transaction', -COMMENT COLUMN to 'The address of the account that is the transaction recipient', -COMMENT COLUMN nonce 'The nonce of the sender account at the time of the transaction', -COMMENT COLUMN gas_price 'The gas price of the transaction in wei', -COMMENT COLUMN gas 'The maximum gas provided for the transaction execution', -COMMENT COLUMN gas_tip_cap 'The priority fee (tip) the user has set for the transaction', -COMMENT COLUMN gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN value 'The value transferred with the transaction in wei', -COMMENT COLUMN type 'The type of the transaction', -COMMENT COLUMN size 'The size of the transaction data in bytes', -COMMENT COLUMN call_data_size 'The size of the call data of the transaction in bytes', -COMMENT COLUMN blob_gas 'The maximum gas provided for the blob transaction execution', -COMMENT COLUMN blob_gas_fee_cap 'The max fee the user has set for the transaction', -COMMENT COLUMN blob_hashes 'The hashes of the blob commitments for blob transactions', -COMMENT COLUMN blob_sidecars_size 'The total size of the sidecars for blob transactions in bytes', -COMMENT COLUMN blob_sidecars_empty_size 'The total empty size of the sidecars for blob transactions in bytes', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_execution_fork_id_hash 'The hash of the fork ID of the current Ethereum network', -COMMENT COLUMN meta_execution_fork_id_next 'The fork ID of the next planned Ethereum network upgrade', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE tmp.mempool_transaction ON CLUSTER '{cluster}' AS tmp.mempool_transaction_local -ENGINE = Distributed('{cluster}', tmp, mempool_transaction_local, unique_key); - -INSERT INTO tmp.mempool_transaction -SELECT - toInt64(cityHash64(hash || from || to || toString(nonce) || toString(type) || meta_client_name) - 9223372036854775808), - NOW(), - * -FROM default.mempool_transaction_local; - -DROP TABLE IF EXISTS default.mempool_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.mempool_transaction_local AND tmp.mempool_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.mempool_transaction ON CLUSTER '{cluster}' AS default.mempool_transaction_local -ENGINE = Distributed('{cluster}', default, mempool_transaction_local, unique_key); - -DROP TABLE IF EXISTS tmp.mempool_transaction ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tmp.mempool_transaction_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/037_libp2p_gossipsub_blob_sidecar.down.sql b/deploy/migrations/clickhouse/037_libp2p_gossipsub_blob_sidecar.down.sql deleted file mode 100644 index 14b2d5e9..00000000 --- a/deploy/migrations/clickhouse/037_libp2p_gossipsub_blob_sidecar.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}'; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/037_libp2p_gossipsub_blob_sidecar.up.sql b/deploy/migrations/clickhouse/037_libp2p_gossipsub_blob_sidecar.up.sql deleted file mode 100644 index 9e277fd4..00000000 --- a/deploy/migrations/clickhouse/037_libp2p_gossipsub_blob_sidecar.up.sql +++ /dev/null @@ -1,88 +0,0 @@ -CREATE TABLE libp2p_gossipsub_blob_sidecar_local -ON CLUSTER '{cluster}' ( - unique_key Int64, - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - wallclock_slot UInt32 Codec(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - wallclock_epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - wallclock_epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 Codec(ZSTD(1)), - proposer_index UInt32 CODEC(ZSTD(1)), - blob_index UInt32 CODEC(ZSTD(1)), - peer_id_unique_key Int64, - message_id String CODEC(ZSTD(1)), - message_size UInt32 Codec(ZSTD(1)), - topic_layer LowCardinality(String), - topic_fork_digest_value LowCardinality(String), - topic_name LowCardinality(String), - topic_encoding LowCardinality(String), - meta_client_name LowCardinality(String), - meta_client_id String Codec(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) Codec(ZSTD(1)), - meta_network_id Int32 Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, unique_key, meta_network_name, meta_client_name); - -ALTER TABLE libp2p_gossipsub_blob_sidecar_local -ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub blob sidecar data', -COMMENT COLUMN unique_key 'Unique identifier for each record', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the event with millisecond precision', -COMMENT COLUMN slot 'Slot number associated with the event', -COMMENT COLUMN slot_start_date_time 'Start date and time of the slot', -COMMENT COLUMN epoch 'Epoch number associated with the event', -COMMENT COLUMN epoch_start_date_time 'Start date and time of the epoch', -COMMENT COLUMN wallclock_slot 'Slot number of the wall clock when the event was received', -COMMENT COLUMN wallclock_slot_start_date_time 'Start date and time of the wall clock slot when the event was received', -COMMENT COLUMN wallclock_epoch 'Epoch number of the wall clock when the event was received', -COMMENT COLUMN wallclock_epoch_start_date_time 'Start date and time of the wall clock epoch when the event was received', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slot start time for propagation', -COMMENT COLUMN proposer_index 'The proposer index of the beacon block', -COMMENT COLUMN blob_index 'Blob index associated with the record', -COMMENT COLUMN peer_id_unique_key 'Unique key associated with the identifier of the peer', -COMMENT COLUMN message_id 'Identifier of the message', -COMMENT COLUMN message_size 'Size of the message in bytes', -COMMENT COLUMN topic_layer 'Layer of the topic in the gossipsub protocol', -COMMENT COLUMN topic_fork_digest_value 'Fork digest value of the topic', -COMMENT COLUMN topic_name 'Name of the topic', -COMMENT COLUMN topic_encoding 'Encoding used for the topic', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Network ID associated with the client', -COMMENT COLUMN meta_network_name 'Name of the network associated with the client'; - -CREATE TABLE libp2p_gossipsub_blob_sidecar -ON CLUSTER '{cluster}' -AS libp2p_gossipsub_blob_sidecar_local -ENGINE = Distributed('{cluster}', default, libp2p_gossipsub_blob_sidecar_local, unique_key); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/038_libp2p_gossipsub_blob_sidecar_block_fields.down.sql b/deploy/migrations/clickhouse/038_libp2p_gossipsub_blob_sidecar_block_fields.down.sql deleted file mode 100644 index adb64a28..00000000 --- a/deploy/migrations/clickhouse/038_libp2p_gossipsub_blob_sidecar_block_fields.down.sql +++ /dev/null @@ -1,11 +0,0 @@ -ALTER TABLE libp2p_gossipsub_blob_sidecar_local on cluster '{cluster}' - DROP COLUMN parent_root; - -ALTER TABLE libp2p_gossipsub_blob_sidecar on cluster '{cluster}' - DROP COLUMN parent_root; - -ALTER TABLE libp2p_gossipsub_blob_sidecar_local on cluster '{cluster}' - DROP COLUMN state_root; - -ALTER TABLE libp2p_gossipsub_blob_sidecar on cluster '{cluster}' - DROP COLUMN state_root; diff --git a/deploy/migrations/clickhouse/038_libp2p_gossipsub_blob_sidecar_block_fields.up.sql b/deploy/migrations/clickhouse/038_libp2p_gossipsub_blob_sidecar_block_fields.up.sql deleted file mode 100644 index 3418f78c..00000000 --- a/deploy/migrations/clickhouse/038_libp2p_gossipsub_blob_sidecar_block_fields.up.sql +++ /dev/null @@ -1,23 +0,0 @@ -ALTER TABLE libp2p_gossipsub_blob_sidecar_local on cluster '{cluster}' -ADD COLUMN parent_root FixedString(66) Codec(ZSTD(1)) AFTER blob_index; - -ALTER TABLE libp2p_gossipsub_blob_sidecar on cluster '{cluster}' -ADD COLUMN parent_root FixedString(66) Codec(ZSTD(1)) AFTER blob_index; - -ALTER TABLE libp2p_gossipsub_blob_sidecar_local on cluster '{cluster}' -COMMENT COLUMN parent_root 'Parent root of the beacon block'; - -ALTER TABLE libp2p_gossipsub_blob_sidecar on cluster '{cluster}' -COMMENT COLUMN parent_root 'Parent root of the beacon block'; - -ALTER TABLE libp2p_gossipsub_blob_sidecar_local on cluster '{cluster}' -ADD COLUMN state_root FixedString(66) Codec(ZSTD(1)) AFTER parent_root; - -ALTER TABLE libp2p_gossipsub_blob_sidecar on cluster '{cluster}' -ADD COLUMN state_root FixedString(66) Codec(ZSTD(1)) AFTER parent_root; - -ALTER TABLE libp2p_gossipsub_blob_sidecar_local on cluster '{cluster}' -COMMENT COLUMN state_root 'State root of the beacon block'; - -ALTER TABLE libp2p_gossipsub_blob_sidecar on cluster '{cluster}' -COMMENT COLUMN state_root 'State root of the beacon block'; diff --git a/deploy/migrations/clickhouse/039_canonical_beacon_validators.down.sql b/deploy/migrations/clickhouse/039_canonical_beacon_validators.down.sql deleted file mode 100644 index 9c64079f..00000000 --- a/deploy/migrations/clickhouse/039_canonical_beacon_validators.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_validators on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_validators_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/039_canonical_beacon_validators.up.sql b/deploy/migrations/clickhouse/039_canonical_beacon_validators.up.sql deleted file mode 100644 index 32067185..00000000 --- a/deploy/migrations/clickhouse/039_canonical_beacon_validators.up.sql +++ /dev/null @@ -1,85 +0,0 @@ -CREATE TABLE default.canonical_beacon_validators_local on cluster '{cluster}' -( - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - `index` UInt32 CODEC(ZSTD(1)), - balance UInt64 CODEC(ZSTD(1)), - `status` LowCardinality(String), - pubkey String CODEC(ZSTD(1)), - withdrawal_credentials String CODEC(ZSTD(1)), - effective_balance UInt64 CODEC(ZSTD(1)), - slashed Bool, - activation_epoch UInt64 CODEC(ZSTD(1)), - activation_eligibility_epoch UInt64 CODEC(ZSTD(1)), - exit_epoch UInt64 CODEC(ZSTD(1)), - withdrawable_epoch UInt64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, index, meta_network_name); - -ALTER TABLE default.canonical_beacon_validators_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a validator state for an epoch.', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN `index` 'The index of the validator', -COMMENT COLUMN `balance` 'The balance of the validator', -COMMENT COLUMN `status` 'The status of the validator', -COMMENT COLUMN pubkey 'The public key of the validator', -COMMENT COLUMN withdrawal_credentials 'The withdrawal credentials of the validator', -COMMENT COLUMN effective_balance 'The effective balance of the validator', -COMMENT COLUMN slashed 'Whether the validator is slashed', -COMMENT COLUMN activation_epoch 'The epoch when the validator was activated', -COMMENT COLUMN activation_eligibility_epoch 'The epoch when the validator was activated', -COMMENT COLUMN exit_epoch 'The epoch when the validator exited', -COMMENT COLUMN withdrawable_epoch 'The epoch when the validator can withdraw', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_validators on cluster '{cluster}' AS canonical_beacon_validators_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_validators_local, cityHash64('epoch_start_date_time', 'index', 'meta_network_name')); diff --git a/deploy/migrations/clickhouse/040_canonical_beacon_validators_split.down.sql b/deploy/migrations/clickhouse/040_canonical_beacon_validators_split.down.sql deleted file mode 100644 index d79fd523..00000000 --- a/deploy/migrations/clickhouse/040_canonical_beacon_validators_split.down.sql +++ /dev/null @@ -1,94 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_validators on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_validators_local on cluster '{cluster}' SYNC; - -DROP TABLE IF EXISTS canonical_beacon_validators_pubkeys on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_validators_pubkeys_local on cluster '{cluster}' SYNC; - -DROP TABLE IF EXISTS canonical_beacon_validators_withdrawal_credentials on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_validators_withdrawal_credentials_local on cluster '{cluster}' SYNC; - -CREATE TABLE default.canonical_beacon_validators_local on cluster '{cluster}' -( - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - `index` UInt32 CODEC(ZSTD(1)), - balance UInt64 CODEC(ZSTD(1)), - `status` LowCardinality(String), - pubkey String CODEC(ZSTD(1)), - withdrawal_credentials String CODEC(ZSTD(1)), - effective_balance UInt64 CODEC(ZSTD(1)), - slashed Bool, - activation_epoch UInt64 CODEC(ZSTD(1)), - activation_eligibility_epoch UInt64 CODEC(ZSTD(1)), - exit_epoch UInt64 CODEC(ZSTD(1)), - withdrawable_epoch UInt64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, index, meta_network_name); - -ALTER TABLE default.canonical_beacon_validators_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a validator state for an epoch.', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN `index` 'The index of the validator', -COMMENT COLUMN `balance` 'The balance of the validator', -COMMENT COLUMN `status` 'The status of the validator', -COMMENT COLUMN pubkey 'The public key of the validator', -COMMENT COLUMN withdrawal_credentials 'The withdrawal credentials of the validator', -COMMENT COLUMN effective_balance 'The effective balance of the validator', -COMMENT COLUMN slashed 'Whether the validator is slashed', -COMMENT COLUMN activation_epoch 'The epoch when the validator was activated', -COMMENT COLUMN activation_eligibility_epoch 'The epoch when the validator was activated', -COMMENT COLUMN exit_epoch 'The epoch when the validator exited', -COMMENT COLUMN withdrawable_epoch 'The epoch when the validator can withdraw', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_validators on cluster '{cluster}' AS canonical_beacon_validators_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_validators_local, cityHash64(epoch_start_date_time, `index`, meta_network_name)); diff --git a/deploy/migrations/clickhouse/040_canonical_beacon_validators_split.up.sql b/deploy/migrations/clickhouse/040_canonical_beacon_validators_split.up.sql deleted file mode 100644 index d6fbcb0c..00000000 --- a/deploy/migrations/clickhouse/040_canonical_beacon_validators_split.up.sql +++ /dev/null @@ -1,226 +0,0 @@ -DROP TABLE IF EXISTS canonical_beacon_validators on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS canonical_beacon_validators_local on cluster '{cluster}' SYNC; - -CREATE TABLE default.canonical_beacon_validators_local on cluster '{cluster}' -( - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - `index` UInt32 CODEC(ZSTD(1)), - balance UInt64 CODEC(ZSTD(1)), - `status` LowCardinality(String), - effective_balance UInt64 CODEC(ZSTD(1)), - slashed Bool, - activation_epoch UInt64 CODEC(ZSTD(1)), - activation_eligibility_epoch UInt64 CODEC(ZSTD(1)), - exit_epoch UInt64 CODEC(ZSTD(1)), - withdrawable_epoch UInt64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, index, meta_network_name); - -ALTER TABLE default.canonical_beacon_validators_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a validator state for an epoch.', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN `index` 'The index of the validator', -COMMENT COLUMN balance 'The balance of the validator', -COMMENT COLUMN `status` 'The status of the validator', -COMMENT COLUMN effective_balance 'The effective balance of the validator', -COMMENT COLUMN slashed 'Whether the validator is slashed', -COMMENT COLUMN activation_epoch 'The epoch when the validator was activated', -COMMENT COLUMN activation_eligibility_epoch 'The epoch when the validator was activated', -COMMENT COLUMN exit_epoch 'The epoch when the validator exited', -COMMENT COLUMN withdrawable_epoch 'The epoch when the validator can withdraw', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_validators on cluster '{cluster}' AS canonical_beacon_validators_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_validators_local, cityHash64(epoch_start_date_time, `index`, meta_network_name)); - -CREATE TABLE default.canonical_beacon_validators_pubkeys_local on cluster '{cluster}' -( - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - -- ensure the first epoch the pubkey was seen is in this table - -- 4294967295 = UInt32 max - `version` UInt32 DEFAULT 4294967295 - toUnixTimestamp(epoch_start_date_time) CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - `index` UInt32 CODEC(ZSTD(1)), - pubkey String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', `version`) -ORDER BY (index, pubkey, meta_network_name); - -ALTER TABLE default.canonical_beacon_validators_pubkeys_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a validator state for an epoch.', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN `version` 'Version of this row, to help with de-duplication we want the latest updated_date_time but earliest epoch_start_date_time the pubkey was seen', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN `index` 'The index of the validator', -COMMENT COLUMN pubkey 'The public key of the validator', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_validators_pubkeys on cluster '{cluster}' AS canonical_beacon_validators_pubkeys_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_validators_pubkeys_local, cityHash64(`index`, meta_network_name)); - -CREATE TABLE default.canonical_beacon_validators_withdrawal_credentials_local on cluster '{cluster}' -( - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - -- ensure the first epoch the withdrawal_credentials was seen is in this table - -- 4294967295 = UInt32 max - `version` UInt32 DEFAULT 4294967295 - toUnixTimestamp(epoch_start_date_time) CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - `index` UInt32 CODEC(ZSTD(1)), - withdrawal_credentials String CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', `version`) -ORDER BY (index, withdrawal_credentials, meta_network_name); - -ALTER TABLE default.canonical_beacon_validators_withdrawal_credentials_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a validator state for an epoch.', -COMMENT COLUMN updated_date_time 'When this row was last updated', -COMMENT COLUMN `version` 'Version of this row, to help with de-duplication we want the latest updated_date_time but earliest epoch_start_date_time the withdrawal_credentials was seen', -COMMENT COLUMN event_date_time 'When the client fetched the beacon block from a beacon node', -COMMENT COLUMN epoch 'The epoch number from beacon block payload', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN `index` 'The index of the validator', -COMMENT COLUMN withdrawal_credentials 'The withdrawal credentials of the validator', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE canonical_beacon_validators_withdrawal_credentials on cluster '{cluster}' AS canonical_beacon_validators_withdrawal_credentials_local -ENGINE = Distributed('{cluster}', default, canonical_beacon_validators_withdrawal_credentials_local, cityHash64(`index`, meta_network_name)); diff --git a/deploy/migrations/clickhouse/041_remove_unique_field.down.sql b/deploy/migrations/clickhouse/041_remove_unique_field.down.sql deleted file mode 100644 index c7a2b032..00000000 --- a/deploy/migrations/clickhouse/041_remove_unique_field.down.sql +++ /dev/null @@ -1,4594 +0,0 @@ --- beacon_api_eth_v1_beacon_committee -CREATE TABLE tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API committee payload', - `validators` Array(UInt32) COMMENT 'The validator indices in the beacon API committee payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_beacon_committee_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_beacon_committee_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_beacon_committee -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - committee_index, - validators, - epoch, - epoch_start_date_time, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_beacon_committee_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_beacon_committee_local -AND tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_beacon_committee_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_beacon_committee_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_blob_sidecar -CREATE TABLE tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `blob_index` UInt64 COMMENT 'The index of blob sidecar in the beacon API event stream payload' CODEC(ZSTD(1)), - `kzg_commitment` FixedString(98) COMMENT 'The KZG commitment in the beacon API event stream payload' CODEC(ZSTD(1)), - `versioned_hash` FixedString(66) COMMENT 'The versioned hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_blob_sidecar_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_blob_sidecar -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - epoch, - epoch_start_date_time, - block_root, - blob_index, - kzg_commitment, - versioned_hash, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_blob_sidecar_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_blob_sidecar_local -AND tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_blob_sidecar_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_block -CREATE TABLE tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `block` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `execution_optimistic` Bool COMMENT 'If the attached beacon node is running in execution optimistic mode', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_block_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_block_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_block -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - block, - epoch, - epoch_start_date_time, - execution_optimistic, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_block_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_block_local -AND tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_block_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_block_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_chain_reorg -CREATE TABLE tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `slot` UInt32 COMMENT 'The slot number of the chain reorg event in the beacon API event stream payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the reorg slot started', - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slots between when the reorg occurred and when the sentry received the event', - `depth` UInt16 COMMENT 'The depth of the chain reorg in the beacon API event stream payload', - `old_head_block` FixedString(66) COMMENT 'The old head block root hash in the beacon API event stream payload', - `new_head_block` FixedString(66) COMMENT 'The new head block root hash in the beacon API event stream payload', - `old_head_state` FixedString(66) COMMENT 'The old head state root hash in the beacon API event stream payload', - `new_head_state` FixedString(66) COMMENT 'The new head state root hash in the beacon API event stream payload', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `execution_optimistic` Bool COMMENT 'Whether the execution of the epoch was optimistic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_chain_reorg_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_chain_reorg_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_chain_reorg -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - old_head_block, - new_head_block - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - depth, - old_head_block, - new_head_block, - old_head_state, - new_head_state, - epoch, - epoch_start_date_time, - execution_optimistic, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_chain_reorg_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_chain_reorg_local -AND tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_chain_reorg_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_chain_reorg_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_contribution_and_proof -CREATE TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `aggregator_index` UInt32 COMMENT 'The validator index of the aggregator in the beacon API event stream payload', - `contribution_slot` UInt32 COMMENT 'The slot number of the contribution in the beacon API event stream payload', - `contribution_slot_start_date_time` DateTime COMMENT 'The wall clock time when the contribution slot started', - `contribution_propagation_slot_start_diff` UInt32 COMMENT 'Difference in slots between when the contribution occurred and when the sentry received the event', - `contribution_beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload', - `contribution_subcommittee_index` LowCardinality(String) COMMENT 'The subcommittee index of the contribution in the beacon API event stream payload', - `contribution_aggregation_bits` String COMMENT 'The aggregation bits of the contribution in the beacon API event stream payload', - `contribution_signature` String COMMENT 'The signature of the contribution in the beacon API event stream payload', - `contribution_epoch` UInt32 COMMENT 'The epoch number of the contribution in the beacon API event stream payload', - `contribution_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the contribution epoch started', - `selection_proof` String COMMENT 'The selection proof in the beacon API event stream payload', - `signature` String COMMENT 'The signature in the beacon API event stream payload', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(contribution_slot_start_date_time) -ORDER BY - ( - contribution_slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_contribution_and_proof_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_contribution_and_proof_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_contribution_and_proof -SELECT - toInt64( - cityHash64( - contribution_slot_start_date_time, - meta_network_name, - meta_client_name, - contribution_beacon_block_root, - contribution_subcommittee_index - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - aggregator_index, - contribution_slot, - contribution_slot_start_date_time, - contribution_propagation_slot_start_diff, - contribution_beacon_block_root, - contribution_subcommittee_index, - contribution_aggregation_bits, - contribution_signature, - contribution_epoch, - contribution_epoch_start_date_time, - selection_proof, - signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_contribution_and_proof_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_contribution_and_proof_local -AND tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_contribution_and_proof_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_contribution_and_proof_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_finalized_checkpoint -CREATE TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `block` FixedString(66) COMMENT 'The finalized block root hash in the beacon API event stream payload', - `state` FixedString(66) COMMENT 'The finalized state root hash in the beacon API event stream payload', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `execution_optimistic` Bool COMMENT 'Whether the execution of the epoch was optimistic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY - ( - epoch_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_finalized_checkpoint_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_finalized_checkpoint -SELECT - toInt64( - cityHash64( - epoch_start_date_time, - meta_network_name, - meta_client_name, - block, - state - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - block, - state, - epoch, - epoch_start_date_time, - execution_optimistic, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_finalized_checkpoint_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_finalized_checkpoint_local -AND tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_finalized_checkpoint_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_finalized_checkpoint_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_head -CREATE TABLE tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time', - `block` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `epoch_transition` Bool COMMENT 'If the event is an epoch transition', - `execution_optimistic` Bool COMMENT 'If the attached beacon node is running in execution optimistic mode', - `previous_duty_dependent_root` FixedString(66) COMMENT 'The previous duty dependent root in the beacon API event stream payload', - `current_duty_dependent_root` FixedString(66) COMMENT 'The current duty dependent root in the beacon API event stream payload', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_head_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_head_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_head -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block, - previous_duty_dependent_root, - current_duty_dependent_root - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - block, - epoch, - epoch_start_date_time, - epoch_transition, - execution_optimistic, - previous_duty_dependent_root, - current_duty_dependent_root, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_head_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_head_local -AND tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_head_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_head_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_voluntary_exit -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `validator_index` UInt32 COMMENT 'The index of the validator making the voluntary exit', - `signature` String COMMENT 'The signature of the voluntary exit in the beacon API event stream payload', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY - ( - epoch_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_voluntary_exit_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_voluntary_exit -SELECT - toInt64( - cityHash64( - epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - epoch, - epoch_start_date_time, - validator_index, - signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_voluntary_exit_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_voluntary_exit_local -AND tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_voluntary_exit_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_proposer_duty -CREATE TABLE tmp.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `proposer_validator_index` UInt32 COMMENT 'The validator index from the proposer duty payload' CODEC(ZSTD(1)), - `proposer_pubkey` String COMMENT 'The BLS public key of the validator from the proposer duty payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains a proposer duty from a beacon block.'; - -CREATE TABLE tmp.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_proposer_duty_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_proposer_duty_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_proposer_duty -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - proposer_validator_index - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - proposer_validator_index, - proposer_pubkey, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_proposer_duty_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_proposer_duty_local -AND tmp.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_proposer_duty_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_proposer_duty_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_validator_attestation_data -CREATE TABLE tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `slot` UInt32 COMMENT 'Slot number in the beacon API validator attestation data payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', - `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API validator attestation data payload', - `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API validator attestation data payload', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API validator attestation data payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `source_epoch` UInt32 COMMENT 'The source epoch number in the beacon API validator attestation data payload', - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started', - `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the beacon API validator attestation data payload', - `target_epoch` UInt32 COMMENT 'The target epoch number in the beacon API validator attestation data payload', - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started', - `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the beacon API validator attestation data payload', - `request_date_time` DateTime COMMENT 'When the request was sent to the beacon node', - `request_duration` UInt32 COMMENT 'The request duration in milliseconds', - `request_slot_start_diff` UInt32 COMMENT 'The difference between the request_date_time and the slot_start_date_time', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API validator attestation data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_validator_attestation_data_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_validator_attestation_data_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v1_validator_attestation_data -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - beacon_block_root, - source_root, - target_root - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - committee_index, - beacon_block_root, - epoch, - epoch_start_date_time, - source_epoch, - source_epoch_start_date_time, - source_root, - target_epoch, - target_epoch_start_date_time, - target_root, - request_date_time, - request_duration, - request_slot_start_diff, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_validator_attestation_data_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_validator_attestation_data_local -AND tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_validator_attestation_data_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_validator_attestation_data_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v2_beacon_block -CREATE TABLE tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry fetched the beacon block from a beacon node', - `slot` UInt32 COMMENT 'The slot number from beacon block payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the reorg slot started', - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block', - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `block_total_bytes` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload', - `block_total_bytes_compressed` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload when compressed using snappy', - `parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block', - `state_root` FixedString(66) COMMENT 'The root hash of the beacon state at this block', - `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block', - `eth1_data_block_hash` FixedString(66) COMMENT 'The block hash of the associated execution block', - `eth1_data_deposit_root` FixedString(66) COMMENT 'The root of the deposit tree in the associated execution block', - `execution_payload_block_hash` FixedString(66) COMMENT 'The block hash of the execution payload', - `execution_payload_block_number` UInt32 COMMENT 'The block number of the execution payload', - `execution_payload_fee_recipient` String COMMENT 'The recipient of the fee for this execution payload', - `execution_payload_state_root` FixedString(66) COMMENT 'The state root of the execution payload', - `execution_payload_parent_hash` FixedString(66) COMMENT 'The parent hash of the execution payload', - `execution_payload_transactions_count` Nullable(UInt32) COMMENT 'The transaction count of the execution payload', - `execution_payload_transactions_total_bytes` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload', - `execution_payload_transactions_total_bytes_compressed` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload when compressed using snappy', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v2_beacon_block_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v2_beacon_block_local, - unique_key -); - -INSERT INTO - tmp.beacon_api_eth_v2_beacon_block -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - parent_root, - state_root - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - block_total_bytes, - block_total_bytes_compressed, - parent_root, - state_root, - proposer_index, - eth1_data_block_hash, - eth1_data_deposit_root, - execution_payload_block_hash, - execution_payload_block_number, - execution_payload_fee_recipient, - execution_payload_state_root, - execution_payload_parent_hash, - execution_payload_transactions_count, - execution_payload_transactions_total_bytes, - execution_payload_transactions_total_bytes_compressed, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v2_beacon_block_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v2_beacon_block_local -AND tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v2_beacon_block_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v2_beacon_block_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' SYNC; - --- beacon_block_classification -CREATE TABLE tmp.beacon_block_classification_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block classification', - `slot` UInt32 COMMENT 'The slot number from beacon block classification', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block classification' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `best_guess_single` LowCardinality(String) COMMENT 'The best guess of the client that generated the beacon block', - `best_guess_multi` LowCardinality(String) COMMENT 'The best guess of the clients that generated the beacon block. This value will typically equal the best_guess_single value, but when multiple clients have high probabilities, this value will have multiple eg. "prysm or lighthouse"', - `client_probability_uncertain` Float32 COMMENT 'The probability that the client that generated the beacon block is uncertain' CODEC(ZSTD(1)), - `client_probability_prysm` Float32 COMMENT 'The probability that the client that generated the beacon block is Prysm' CODEC(ZSTD(1)), - `client_probability_teku` Float32 COMMENT 'The probability that the client that generated the beacon block is Teku' CODEC(ZSTD(1)), - `client_probability_nimbus` Float32 COMMENT 'The probability that the client that generated the beacon block is Nimbus' CODEC(ZSTD(1)), - `client_probability_lodestar` Float32 COMMENT 'The probability that the client that generated the beacon block is Lodestar' CODEC(ZSTD(1)), - `client_probability_grandine` Float32 COMMENT 'The probability that the client that generated the beacon block is Grandine' CODEC(ZSTD(1)), - `client_probability_lighthouse` Float32 COMMENT 'The probability that the client that generated the beacon block is Lighthouse' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon block classification for a given slot. This is a best guess based on the client probabilities of the proposer. This is not guaranteed to be correct.'; - -CREATE TABLE tmp.beacon_block_classification ON CLUSTER '{cluster}' AS tmp.beacon_block_classification_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_block_classification_local, - unique_key -); - -INSERT INTO - tmp.beacon_block_classification -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - proposer_index - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - best_guess_single, - best_guess_multi, - client_probability_uncertain, - client_probability_prysm, - client_probability_teku, - client_probability_nimbus, - client_probability_lodestar, - client_probability_grandine, - client_probability_lighthouse, - proposer_index, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_block_classification_local; - -DROP TABLE IF EXISTS default.beacon_block_classification ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_block_classification_local -AND tmp.beacon_block_classification_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_block_classification ON CLUSTER '{cluster}' AS default.beacon_block_classification_local ENGINE = Distributed( - '{cluster}', - default, - beacon_block_classification_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_block_classification ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_block_classification_local ON CLUSTER '{cluster}' SYNC; - --- beacon_p2p_attestation -CREATE TABLE tmp.beacon_p2p_attestation_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node', - `slot` UInt32 COMMENT 'Slot number in the beacon P2P payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon P2P payload', - `attesting_validator_index` Nullable(UInt32) COMMENT 'The index of the validator attesting to the event' CODEC(ZSTD(1)), - `attesting_validator_committee_index` LowCardinality(String) COMMENT 'The committee index of the attesting validator', - `aggregation_bits` String COMMENT 'The aggregation bits of the event in the beacon P2P payload' CODEC(ZSTD(1)), - `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch` UInt32 COMMENT 'The source epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `target_epoch` UInt32 COMMENT 'The target epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), - `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `attestation_subnet` LowCardinality(String) COMMENT 'The attestation subnet the attestation was gossiped on', - `validated` Bool COMMENT 'Whether the attestation was validated by the client', - `peer_id` String COMMENT 'The originating peer ID for the gossiped data' CODEC(ZSTD(1)), - `peer_latency` UInt32 COMMENT 'The latency of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_version` LowCardinality(String) COMMENT 'Peer client version that gossiped the data', - `peer_version_major` LowCardinality(String) COMMENT 'Peer client major version that gossiped the data', - `peer_version_minor` LowCardinality(String) COMMENT 'Peer client minor version that gossiped the data', - `peer_version_patch` LowCardinality(String) COMMENT 'Peer client patch version that gossiped the data', - `peer_implementation` LowCardinality(String) COMMENT 'Peer client implementation that gossiped the data', - `peer_ip` Nullable(IPv6) COMMENT 'IP address of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_city` LowCardinality(String) COMMENT 'City of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_country` LowCardinality(String) COMMENT 'Country of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_country_code` LowCardinality(String) COMMENT 'Country code of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the peer that gossiped the data' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains beacon chain P2P "attestation" data'; - -CREATE TABLE tmp.beacon_p2p_attestation ON CLUSTER '{cluster}' AS tmp.beacon_p2p_attestation_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_p2p_attestation_local, - unique_key -); - -INSERT INTO - tmp.beacon_p2p_attestation -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - beacon_block_root, - source_root, - target_root, - attestation_subnet, - peer_id - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - committee_index, - attesting_validator_index, - attesting_validator_committee_index, - aggregation_bits, - beacon_block_root, - epoch, - epoch_start_date_time, - source_epoch, - source_epoch_start_date_time, - source_root, - target_epoch, - target_epoch_start_date_time, - target_root, - attestation_subnet, - validated, - peer_id, - peer_latency, - peer_version, - peer_version_major, - peer_version_minor, - peer_version_patch, - peer_implementation, - peer_ip, - peer_geo_city, - peer_geo_country, - peer_geo_country_code, - peer_geo_continent_code, - peer_geo_longitude, - peer_geo_latitude, - peer_geo_autonomous_system_number, - peer_geo_autonomous_system_organization, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_p2p_attestation_local; - -DROP TABLE IF EXISTS default.beacon_p2p_attestation ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_p2p_attestation_local -AND tmp.beacon_p2p_attestation_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_p2p_attestation ON CLUSTER '{cluster}' AS default.beacon_p2p_attestation_local ENGINE = Distributed( - '{cluster}', - default, - beacon_p2p_attestation_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.beacon_p2p_attestation ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_p2p_attestation_local ON CLUSTER '{cluster}' SYNC; - --- block_native_mempool_transaction -CREATE TABLE tmp.block_native_mempool_transaction_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `detecttime` DateTime64(3) COMMENT 'Timestamp that the transaction was detected in mempool' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'Unique identifier hash for a given transaction' CODEC(ZSTD(1)), - `status` LowCardinality(String) COMMENT 'Status of the transaction', - `region` LowCardinality(String) COMMENT 'The geographic region for the node that detected the transaction', - `reorg` Nullable(FixedString(66)) COMMENT 'If there was a reorg, refers to the blockhash of the reorg' CODEC(ZSTD(1)), - `replace` Nullable(FixedString(66)) COMMENT 'If the transaction was replaced (speedup/cancel), the transaction hash of the replacement' CODEC(ZSTD(1)), - `curblocknumber` Nullable(UInt64) COMMENT 'The block number the event was detected in' CODEC(ZSTD(1)), - `failurereason` Nullable(String) COMMENT 'If a transaction failed, this field provides contextual information' CODEC(ZSTD(1)), - `blockspending` Nullable(UInt64) COMMENT 'If a transaction was finalized (confirmed, failed), this refers to the number of blocks that the transaction was waiting to get on-chain' CODEC(ZSTD(1)), - `timepending` Nullable(UInt64) COMMENT 'If a transaction was finalized (confirmed, failed), this refers to the time in milliseconds that the transaction was waiting to get on-chain' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'A unique number which counts the number of transactions sent from a given address' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum number of gas units allowed for the transaction' CODEC(ZSTD(1)), - `gasprice` UInt128 COMMENT 'The price offered to the miner/validator per unit of gas. Denominated in wei' CODEC(ZSTD(1)), - `value` UInt128 COMMENT 'The amount of ETH transferred or sent to contract. Denominated in wei' CODEC(ZSTD(1)), - `toaddress` Nullable(FixedString(42)) COMMENT 'The destination of a given transaction' CODEC(ZSTD(1)), - `fromaddress` FixedString(42) COMMENT 'The source/initiator of a given transaction' CODEC(ZSTD(1)), - `datasize` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `data4bytes` Nullable(FixedString(10)) COMMENT 'The first 4 bytes of the call data of the transaction' CODEC(ZSTD(1)), - `network` LowCardinality(String) COMMENT 'The specific Ethereum network used', - `type` UInt8 COMMENT '"Post EIP-1559, this indicates how the gas parameters are submitted to the network: - type 0 - legacy - type 1 - usage of access lists according to EIP-2930 - type 2 - using maxpriorityfeepergas and maxfeepergas"' CODEC(ZSTD(1)), - `maxpriorityfeepergas` Nullable(UInt128) COMMENT 'The maximum value for a tip offered to the miner/validator per unit of gas. The actual tip paid can be lower if (maxfee - basefee) < maxpriorityfee. Denominated in wei' CODEC(ZSTD(1)), - `maxfeepergas` Nullable(UInt128) COMMENT 'The maximum value for the transaction fee (including basefee and tip) offered to the miner/validator per unit of gas. Denominated in wei' CODEC(ZSTD(1)), - `basefeepergas` Nullable(UInt128) COMMENT 'The fee per unit of gas paid and burned for the curblocknumber. This fee is algorithmically determined. Denominated in wei' CODEC(ZSTD(1)), - `dropreason` Nullable(String) COMMENT 'If the transaction was dropped from the mempool, this describes the contextual reason for the drop' CODEC(ZSTD(1)), - `rejectionreason` Nullable(String) COMMENT 'If the transaction was rejected from the mempool, this describes the contextual reason for the rejection' CODEC(ZSTD(1)), - `stuck` Bool COMMENT 'A transaction was detected in the queued area of the mempool and is not eligible for inclusion in a block' CODEC(ZSTD(1)), - `gasused` Nullable(UInt64) COMMENT 'If the transaction was published on-chain, this value indicates the amount of gas that was actually consumed. Denominated in wei' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(detecttime) -ORDER BY - ( - detecttime, - unique_key, - network - ) COMMENT 'Contains transactions from block native mempool dataset'; - -CREATE TABLE tmp.block_native_mempool_transaction ON CLUSTER '{cluster}' AS tmp.block_native_mempool_transaction_local ENGINE = Distributed( - '{cluster}', - tmp, - block_native_mempool_transaction_local, - unique_key -); - -INSERT INTO - tmp.block_native_mempool_transaction -SELECT - toInt64( - cityHash64( - detecttime, - network, - hash, - fromaddress, - nonce, - gas - ) - 9223372036854775808 - ), - NOW(), - detecttime, - hash, - status, - region, - reorg, - replace, - curblocknumber, - failurereason, - blockspending, - timepending, - nonce, - gas, - gasprice, - value, - toaddress, - fromaddress, - datasize, - data4bytes, - network, - type, - maxpriorityfeepergas, - maxfeepergas, - basefeepergas, - dropreason, - rejectionreason, - stuck, - gasused -FROM - default.block_native_mempool_transaction_local; - -DROP TABLE IF EXISTS default.block_native_mempool_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.block_native_mempool_transaction_local -AND tmp.block_native_mempool_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.block_native_mempool_transaction ON CLUSTER '{cluster}' AS default.block_native_mempool_transaction_local ENGINE = Distributed( - '{cluster}', - default, - block_native_mempool_transaction_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.block_native_mempool_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.block_native_mempool_transaction_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_blob_sidecar -CREATE TABLE tmp.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block' CODEC(ZSTD(1)), - `versioned_hash` FixedString(66) COMMENT 'The versioned hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `kzg_commitment` FixedString(98) COMMENT 'The KZG commitment in the blob sidecar payload' CODEC(ZSTD(1)), - `kzg_proof` FixedString(98) COMMENT 'The KZG proof in the blob sidecar payload' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), - `blob_index` UInt64 COMMENT 'The index of blob sidecar in the blob sidecar payload' CODEC(ZSTD(1)), - `blob_size` UInt32 COMMENT 'The total bytes of the blob' CODEC(ZSTD(1)), - `blob_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the blob in bytes' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains a blob sidecar from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' AS tmp.canonical_beacon_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_blob_sidecar_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_blob_sidecar -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - blob_index - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_parent_root, - versioned_hash, - kzg_commitment, - kzg_proof, - proposer_index, - blob_index, - blob_size, - blob_empty_size, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_blob_sidecar_local; - -DROP TABLE IF EXISTS default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_blob_sidecar_local -AND tmp.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' AS default.canonical_beacon_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_blob_sidecar_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_attester_slashing -CREATE TABLE tmp.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `attestation_1_attesting_indices` Array(UInt32) COMMENT 'The attesting indices from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_signature` String COMMENT 'The signature from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_data_beacon_block_root` FixedString(66) COMMENT 'The beacon block root from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_data_slot` UInt32 COMMENT 'The slot number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_1_data_index` UInt32 COMMENT 'The attestor index from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_data_source_epoch` UInt32 COMMENT 'The source epoch number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_1_data_source_root` FixedString(66) COMMENT 'The source root from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_data_target_epoch` UInt32 COMMENT 'The target epoch number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_1_data_target_root` FixedString(66) COMMENT 'The target root from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_attesting_indices` Array(UInt32) COMMENT 'The attesting indices from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_signature` String COMMENT 'The signature from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_data_beacon_block_root` FixedString(66) COMMENT 'The beacon block root from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_data_slot` UInt32 COMMENT 'The slot number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_2_data_index` UInt32 COMMENT 'The attestor index from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_data_source_epoch` UInt32 COMMENT 'The source epoch number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_2_data_source_root` FixedString(66) COMMENT 'The source root from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_data_target_epoch` UInt32 COMMENT 'The target epoch number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_2_data_target_root` FixedString(66) COMMENT 'The target root from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains attester slashing from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_attester_slashing_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_attester_slashing_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_block_attester_slashing -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - attestation_1_attesting_indices, - attestation_2_attesting_indices, - attestation_1_data_slot, - attestation_2_data_slot, - attestation_1_data_beacon_block_root, - attestation_2_data_beacon_block_root - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - attestation_1_attesting_indices, - attestation_1_signature, - attestation_1_data_beacon_block_root, - attestation_1_data_slot, - attestation_1_data_index, - attestation_1_data_source_epoch, - attestation_1_data_source_root, - attestation_1_data_target_epoch, - attestation_1_data_target_root, - attestation_2_attesting_indices, - attestation_2_signature, - attestation_2_data_beacon_block_root, - attestation_2_data_slot, - attestation_2_data_index, - attestation_2_data_source_epoch, - attestation_2_data_source_root, - attestation_2_data_target_epoch, - attestation_2_data_target_root, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_attester_slashing_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_attester_slashing_local -AND tmp.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' AS default.canonical_beacon_block_attester_slashing_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_attester_slashing_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_bls_to_execution_change -CREATE TABLE tmp.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `exchanging_message_validator_index` UInt32 COMMENT 'The validator index from the exchanging message' CODEC(ZSTD(1)), - `exchanging_message_from_bls_pubkey` String COMMENT 'The BLS public key from the exchanging message' CODEC(ZSTD(1)), - `exchanging_message_to_execution_address` FixedString(42) COMMENT 'The execution address from the exchanging message' CODEC(ZSTD(1)), - `exchanging_signature` String COMMENT 'The signature for the exchanging message' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains bls to execution change from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_bls_to_execution_change_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_bls_to_execution_change_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_block_bls_to_execution_change -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - exchanging_message_validator_index, - exchanging_message_from_bls_pubkey, - exchanging_message_to_execution_address - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - exchanging_message_validator_index, - exchanging_message_from_bls_pubkey, - exchanging_message_to_execution_address, - exchanging_signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_bls_to_execution_change_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_bls_to_execution_change_local -AND tmp.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' AS default.canonical_beacon_block_bls_to_execution_change_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_bls_to_execution_change_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_deposit -CREATE TABLE tmp.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `deposit_proof` Array(String) COMMENT 'The proof of the deposit data' CODEC(ZSTD(1)), - `deposit_data_pubkey` String COMMENT 'The BLS public key of the validator from the deposit data' CODEC(ZSTD(1)), - `deposit_data_withdrawal_credentials` FixedString(66) COMMENT 'The withdrawal credentials of the validator from the deposit data' CODEC(ZSTD(1)), - `deposit_data_amount` UInt128 COMMENT 'The amount of the deposit from the deposit data' CODEC(ZSTD(1)), - `deposit_data_signature` String COMMENT 'The signature of the deposit data' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains a deposit from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_deposit ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_deposit_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_deposit_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_block_deposit -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - deposit_data_pubkey, - deposit_proof - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - deposit_proof, - deposit_data_pubkey, - deposit_data_withdrawal_credentials, - deposit_data_amount, - deposit_data_signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_deposit_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_deposit_local -AND tmp.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' AS default.canonical_beacon_block_deposit_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_deposit_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_deposit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_execution_transaction -CREATE TABLE tmp.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `position` UInt32 COMMENT 'The position of the transaction in the beacon block' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), - `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), - `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), - `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), - `gas_tip_cap` Nullable(UInt128) COMMENT 'The priority fee (tip) the user has set for the transaction' CODEC(ZSTD(1)), - `gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), - `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), - `type` UInt8 COMMENT 'The type of the transaction' CODEC(ZSTD(1)), - `size` UInt32 COMMENT 'The size of the transaction data in bytes' CODEC(ZSTD(1)), - `call_data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `blob_gas` Nullable(UInt64) COMMENT 'The maximum gas provided for the blob transaction execution' CODEC(ZSTD(1)), - `blob_gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), - `blob_hashes` Array(String) COMMENT 'The hashes of the blob commitments for blob transactions' CODEC(ZSTD(1)), - `blob_sidecars_size` Nullable(UInt32) COMMENT 'The total size of the sidecars for blob transactions in bytes' CODEC(ZSTD(1)), - `blob_sidecars_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the sidecars for blob transactions in bytes' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains execution transaction from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_execution_transaction_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_execution_transaction_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_block_execution_transaction -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - position, - hash, - nonce - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - position, - hash, -from -, - to, - nonce, - gas_price, - gas, - gas_tip_cap, - gas_fee_cap, - value, - type, - size, - call_data_size, - blob_gas, - blob_gas_fee_cap, - blob_hashes, - blob_sidecars_size, - blob_sidecars_empty_size, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_execution_transaction_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_execution_transaction_local -AND tmp.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' AS default.canonical_beacon_block_execution_transaction_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_execution_transaction_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block -CREATE TABLE tmp.canonical_beacon_block_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `block_total_bytes` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload' CODEC(ZSTD(1)), - `block_total_bytes_compressed` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload when compressed using snappy' CODEC(ZSTD(1)), - `parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block' CODEC(ZSTD(1)), - `state_root` FixedString(66) COMMENT 'The root hash of the beacon state at this block' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), - `eth1_data_block_hash` FixedString(66) COMMENT 'The block hash of the associated execution block' CODEC(ZSTD(1)), - `eth1_data_deposit_root` FixedString(66) COMMENT 'The root of the deposit tree in the associated execution block' CODEC(ZSTD(1)), - `execution_payload_block_hash` FixedString(66) COMMENT 'The block hash of the execution payload' CODEC(ZSTD(1)), - `execution_payload_block_number` UInt32 COMMENT 'The block number of the execution payload' CODEC(DoubleDelta, ZSTD(1)), - `execution_payload_fee_recipient` String COMMENT 'The recipient of the fee for this execution payload' CODEC(ZSTD(1)), - `execution_payload_state_root` FixedString(66) COMMENT 'The state root of the execution payload' CODEC(ZSTD(1)), - `execution_payload_parent_hash` FixedString(66) COMMENT 'The parent hash of the execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_count` Nullable(UInt32) COMMENT 'The transaction count of the execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_total_bytes` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_total_bytes_compressed` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload when compressed using snappy' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains beacon block from a beacon node.'; - -CREATE TABLE tmp.canonical_beacon_block ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_block -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - block_total_bytes, - block_total_bytes_compressed, - parent_root, - state_root, - proposer_index, - eth1_data_block_hash, - eth1_data_deposit_root, - execution_payload_block_hash, - execution_payload_block_number, - execution_payload_fee_recipient, - execution_payload_state_root, - execution_payload_parent_hash, - execution_payload_transactions_count, - execution_payload_transactions_total_bytes, - execution_payload_transactions_total_bytes_compressed, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_local -AND tmp.canonical_beacon_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block ON CLUSTER '{cluster}' AS default.canonical_beacon_block_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_proposer_slashing -CREATE TABLE tmp.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `signed_header_1_message_slot` UInt32 COMMENT 'The slot number from the first signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `signed_header_1_message_proposer_index` UInt32 COMMENT 'The proposer index from the first signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `signed_header_1_message_body_root` FixedString(66) COMMENT 'The body root from the first signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_1_message_parent_root` FixedString(66) COMMENT 'The parent root from the first signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_1_message_state_root` FixedString(66) COMMENT 'The state root from the first signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_1_signature` String COMMENT 'The signature for the first signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_2_message_slot` UInt32 COMMENT 'The slot number from the second signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `signed_header_2_message_proposer_index` UInt32 COMMENT 'The proposer index from the second signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `signed_header_2_message_body_root` FixedString(66) COMMENT 'The body root from the second signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_2_message_parent_root` FixedString(66) COMMENT 'The parent root from the second signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_2_message_state_root` FixedString(66) COMMENT 'The state root from the second signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_2_signature` String COMMENT 'The signature for the second signed header in the slashing payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains proposer slashing from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_proposer_slashing_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_proposer_slashing_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_block_proposer_slashing -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - signed_header_1_message_slot, - signed_header_2_message_slot, - signed_header_1_message_proposer_index, - signed_header_2_message_proposer_index, - signed_header_1_message_body_root, - signed_header_2_message_body_root - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - signed_header_1_message_slot, - signed_header_1_message_proposer_index, - signed_header_1_message_body_root, - signed_header_1_message_parent_root, - signed_header_1_message_state_root, - signed_header_1_signature, - signed_header_2_message_slot, - signed_header_2_message_proposer_index, - signed_header_2_message_body_root, - signed_header_2_message_parent_root, - signed_header_2_message_state_root, - signed_header_2_signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_proposer_slashing_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_proposer_slashing_local -AND tmp.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' AS default.canonical_beacon_block_proposer_slashing_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_proposer_slashing_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_voluntary_exit -CREATE TABLE tmp.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `voluntary_exit_message_epoch` UInt32 COMMENT 'The epoch number from the exit message' CODEC(DoubleDelta, ZSTD(1)), - `voluntary_exit_message_validator_index` UInt32 COMMENT 'The validator index from the exit message' CODEC(ZSTD(1)), - `voluntary_exit_signature` String COMMENT 'The signature of the exit message' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains a voluntary exit from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_voluntary_exit_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_block_voluntary_exit -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - voluntary_exit_message_epoch, - voluntary_exit_message_validator_index - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - voluntary_exit_message_epoch, - voluntary_exit_message_validator_index, - voluntary_exit_signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_voluntary_exit_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_voluntary_exit_local -AND tmp.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' AS default.canonical_beacon_block_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_voluntary_exit_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_withdrawal -CREATE TABLE tmp.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `withdrawal_index` UInt32 COMMENT 'The index of the withdrawal' CODEC(ZSTD(1)), - `withdrawal_validator_index` UInt32 COMMENT 'The validator index from the withdrawal data' CODEC(ZSTD(1)), - `withdrawal_address` FixedString(42) COMMENT 'The address of the account that is the withdrawal recipient' CODEC(ZSTD(1)), - `withdrawal_amount` UInt128 COMMENT 'The amount of the withdrawal from the withdrawal data' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains a withdrawal from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_withdrawal_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_withdrawal_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_block_withdrawal -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - withdrawal_index, - withdrawal_validator_index - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - withdrawal_index, - withdrawal_validator_index, - withdrawal_address, - withdrawal_amount, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_withdrawal_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_withdrawal_local -AND tmp.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' AS default.canonical_beacon_block_withdrawal_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_withdrawal_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_elaborated_attestation -CREATE TABLE tmp.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the elaborated attestation from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `block_slot` UInt32 COMMENT 'The slot number of the block containing the attestation' CODEC(DoubleDelta, ZSTD(1)), - `block_slot_start_date_time` DateTime COMMENT 'The wall clock time when the block slot started' CODEC(DoubleDelta, ZSTD(1)), - `block_epoch` UInt32 COMMENT 'The epoch number of the block containing the attestation' CODEC(DoubleDelta, ZSTD(1)), - `block_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the block epoch started' CODEC(DoubleDelta, ZSTD(1)), - `position_in_block` UInt32 COMMENT 'The position of the attestation in the block' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root of the block containing the attestation' CODEC(ZSTD(1)), - `validators` Array(UInt32) COMMENT 'Array of validator indices participating in the attestation' CODEC(ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The index of the committee making the attestation', - `beacon_block_root` FixedString(66) COMMENT 'The root of the beacon block being attested to' CODEC(ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number being attested to' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), - `source_epoch` UInt32 COMMENT 'The source epoch referenced in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_root` FixedString(66) COMMENT 'The root of the source checkpoint in the attestation' CODEC(ZSTD(1)), - `target_epoch` UInt32 COMMENT 'The target epoch referenced in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), - `target_root` FixedString(66) COMMENT 'The root of the target checkpoint in the attestation' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains elaborated attestations from beacon blocks.'; - -CREATE TABLE tmp.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' AS tmp.canonical_beacon_elaborated_attestation_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_elaborated_attestation_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_elaborated_attestation -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - block_slot, - position_in_block, - beacon_block_root, - slot, - committee_index, - source_root, - target_root - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - block_slot, - block_slot_start_date_time, - block_epoch, - block_epoch_start_date_time, - position_in_block, - block_root, - validators, - committee_index, - beacon_block_root, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - source_epoch, - source_epoch_start_date_time, - source_root, - target_epoch, - target_epoch_start_date_time, - target_root, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_elaborated_attestation_local; - -DROP TABLE IF EXISTS default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_elaborated_attestation_local -AND tmp.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' AS default.canonical_beacon_elaborated_attestation_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_elaborated_attestation_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_proposer_duty -CREATE TABLE tmp.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the proposer duty information from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number for which the proposer duty is assigned' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number containing the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `proposer_validator_index` UInt32 COMMENT 'The validator index of the proposer for the slot' CODEC(ZSTD(1)), - `proposer_pubkey` String COMMENT 'The public key of the validator proposer' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the even' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name - ) COMMENT 'Contains a proposer duty from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' AS tmp.canonical_beacon_proposer_duty_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_proposer_duty_local, - unique_key -); - -INSERT INTO - tmp.canonical_beacon_proposer_duty -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - proposer_validator_index, - proposer_pubkey - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - proposer_validator_index, - proposer_pubkey, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_proposer_duty_local; - -DROP TABLE IF EXISTS default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_proposer_duty_local -AND tmp.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' AS default.canonical_beacon_proposer_duty_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_proposer_duty_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_add_peer -CREATE TABLE tmp.libp2p_add_peer_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `protocol` LowCardinality(String) COMMENT 'Protocol used by the peer', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains the details of the peers added to the libp2p client.'; - -CREATE TABLE tmp.libp2p_add_peer ON CLUSTER '{cluster}' AS tmp.libp2p_add_peer_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_add_peer_local, - unique_key -); - -INSERT INTO - tmp.libp2p_add_peer -SELECT - toInt64( - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - peer_id_unique_key, - protocol, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_add_peer_local; - -DROP TABLE IF EXISTS default.libp2p_add_peer ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_add_peer_local -AND tmp.libp2p_add_peer_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_add_peer ON CLUSTER '{cluster}' AS default.libp2p_add_peer_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_add_peer_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_add_peer ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_add_peer_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_connected -CREATE TABLE tmp.libp2p_connected_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `remote_peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the remote peer', - `remote_protocol` LowCardinality(String) COMMENT 'Protocol of the remote peer', - `remote_transport_protocol` LowCardinality(String) COMMENT 'Transport protocol of the remote peer', - `remote_port` UInt16 COMMENT 'Port of the remote peer' CODEC(ZSTD(1)), - `remote_ip` Nullable(IPv6) COMMENT 'IP address of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_city` LowCardinality(String) COMMENT 'City of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_country` LowCardinality(String) COMMENT 'Country of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_country_code` LowCardinality(String) COMMENT 'Country code of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_agent_implementation` LowCardinality(String) COMMENT 'Implementation of the remote peer', - `remote_agent_version` LowCardinality(String) COMMENT 'Version of the remote peer', - `remote_agent_version_major` LowCardinality(String) COMMENT 'Major version of the remote peer', - `remote_agent_version_minor` LowCardinality(String) COMMENT 'Minor version of the remote peer', - `remote_agent_version_patch` LowCardinality(String) COMMENT 'Patch version of the remote peer', - `remote_agent_platform` LowCardinality(String) COMMENT 'Platform of the remote peer', - `direction` LowCardinality(String) COMMENT 'Connection direction', - `opened` DateTime COMMENT 'Timestamp when the connection was opened' CODEC(DoubleDelta, ZSTD(1)), - `transient` Bool COMMENT 'Whether the connection is transient', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains the details of the CONNECTED events from the libp2p client.'; - -CREATE TABLE tmp.libp2p_connected ON CLUSTER '{cluster}' AS tmp.libp2p_connected_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_connected_local, - unique_key -); - -INSERT INTO - tmp.libp2p_connected -SELECT - toInt64( - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - direction, - opened - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - remote_peer_id_unique_key, - remote_protocol, - remote_transport_protocol, - remote_port, - remote_ip, - remote_geo_city, - remote_geo_country, - remote_geo_country_code, - remote_geo_continent_code, - remote_geo_longitude, - remote_geo_latitude, - remote_geo_autonomous_system_number, - remote_geo_autonomous_system_organization, - remote_agent_implementation, - remote_agent_version, - remote_agent_version_major, - remote_agent_version_minor, - remote_agent_version_patch, - remote_agent_platform, - direction, - opened, - transient, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_connected_local; - -DROP TABLE IF EXISTS default.libp2p_connected ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_connected_local -AND tmp.libp2p_connected_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_connected ON CLUSTER '{cluster}' AS default.libp2p_connected_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_connected_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_connected ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_connected_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_disconnected -CREATE TABLE tmp.libp2p_disconnected_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `remote_peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the remote peer', - `remote_protocol` LowCardinality(String) COMMENT 'Protocol of the remote peer', - `remote_transport_protocol` LowCardinality(String) COMMENT 'Transport protocol of the remote peer', - `remote_port` UInt16 COMMENT 'Port of the remote peer' CODEC(ZSTD(1)), - `remote_ip` Nullable(IPv6) COMMENT 'IP address of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_city` LowCardinality(String) COMMENT 'City of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_country` LowCardinality(String) COMMENT 'Country of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_country_code` LowCardinality(String) COMMENT 'Country code of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_agent_implementation` LowCardinality(String) COMMENT 'Implementation of the remote peer', - `remote_agent_version` LowCardinality(String) COMMENT 'Version of the remote peer', - `remote_agent_version_major` LowCardinality(String) COMMENT 'Major version of the remote peer', - `remote_agent_version_minor` LowCardinality(String) COMMENT 'Minor version of the remote peer', - `remote_agent_version_patch` LowCardinality(String) COMMENT 'Patch version of the remote peer', - `remote_agent_platform` LowCardinality(String) COMMENT 'Platform of the remote peer', - `direction` LowCardinality(String) COMMENT 'Connection direction', - `opened` DateTime COMMENT 'Timestamp when the connection was opened' CODEC(DoubleDelta, ZSTD(1)), - `transient` Bool COMMENT 'Whether the connection is transient', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains the details of the DISCONNECTED events from the libp2p client.'; - -CREATE TABLE tmp.libp2p_disconnected ON CLUSTER '{cluster}' AS tmp.libp2p_disconnected_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_disconnected_local, - unique_key -); - -INSERT INTO - tmp.libp2p_disconnected -SELECT - toInt64( - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - direction, - opened - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - remote_peer_id_unique_key, - remote_protocol, - remote_transport_protocol, - remote_port, - remote_ip, - remote_geo_city, - remote_geo_country, - remote_geo_country_code, - remote_geo_continent_code, - remote_geo_longitude, - remote_geo_latitude, - remote_geo_autonomous_system_number, - remote_geo_autonomous_system_organization, - remote_agent_implementation, - remote_agent_version, - remote_agent_version_major, - remote_agent_version_minor, - remote_agent_version_patch, - remote_agent_platform, - direction, - opened, - transient, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_disconnected_local; - -DROP TABLE IF EXISTS default.libp2p_disconnected ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_disconnected_local -AND tmp.libp2p_disconnected_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_disconnected ON CLUSTER '{cluster}' AS default.libp2p_disconnected_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_disconnected_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_disconnected ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_disconnected_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_gossipsub_beacon_attestation -CREATE TABLE tmp.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The committee index in the attestation', - `attesting_validator_index` Nullable(UInt32) COMMENT 'The index of the validator attesting to the event' CODEC(ZSTD(1)), - `attesting_validator_committee_index` LowCardinality(String) COMMENT 'The committee index of the attesting validator', - `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', - `aggregation_bits` String COMMENT 'The aggregation bits of the event in the attestation' CODEC(ZSTD(1)), - `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the attestation' CODEC(ZSTD(1)), - `source_epoch` UInt32 COMMENT 'The source epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the attestation' CODEC(ZSTD(1)), - `target_epoch` UInt32 COMMENT 'The target epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), - `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the attestation' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Network ID associated with the client' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Table for libp2p gossipsub beacon attestation data.'; - -CREATE TABLE tmp.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' AS tmp.libp2p_gossipsub_beacon_attestation_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_gossipsub_beacon_attestation_local, - unique_key -); - -INSERT INTO - tmp.libp2p_gossipsub_beacon_attestation -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - committee_index, - attesting_validator_index, - attesting_validator_committee_index, - wallclock_slot, - wallclock_slot_start_date_time, - wallclock_epoch, - wallclock_epoch_start_date_time, - propagation_slot_start_diff, - peer_id_unique_key, - message_id, - message_size, - topic_layer, - topic_fork_digest_value, - topic_name, - topic_encoding, - aggregation_bits, - beacon_block_root, - source_epoch, - source_epoch_start_date_time, - source_root, - target_epoch, - target_epoch_start_date_time, - target_root, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_gossipsub_beacon_attestation_local; - -DROP TABLE IF EXISTS default.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_gossipsub_beacon_attestation_local -AND tmp.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' AS default.libp2p_gossipsub_beacon_attestation_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_gossipsub_beacon_attestation_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_gossipsub_beacon_block -CREATE TABLE tmp.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), - `block` FixedString(66) COMMENT 'The beacon block root hash' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The proposer index of the beacon block' CODEC(ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Network ID associated with the client' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Table for libp2p gossipsub beacon block data.'; - -CREATE TABLE tmp.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' AS tmp.libp2p_gossipsub_beacon_block_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_gossipsub_beacon_block_local, - unique_key -); - -INSERT INTO - tmp.libp2p_gossipsub_beacon_block -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - wallclock_slot, - wallclock_slot_start_date_time, - wallclock_epoch, - wallclock_epoch_start_date_time, - propagation_slot_start_diff, - block, - proposer_index, - peer_id_unique_key, - message_id, - message_size, - topic_layer, - topic_fork_digest_value, - topic_name, - topic_encoding, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_gossipsub_beacon_block_local; - -DROP TABLE IF EXISTS default.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_gossipsub_beacon_block_local -AND tmp.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' AS default.libp2p_gossipsub_beacon_block_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_gossipsub_beacon_block_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_gossipsub_blob_sidecar -CREATE TABLE tmp.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The proposer index of the beacon block' CODEC(ZSTD(1)), - `blob_index` UInt32 COMMENT 'Blob index associated with the record' CODEC(ZSTD(1)), - `parent_root` FixedString(66) COMMENT 'Parent root of the beacon block' CODEC(ZSTD(1)), - `state_root` FixedString(66) COMMENT 'State root of the beacon block' CODEC(ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Network ID associated with the client' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Table for libp2p gossipsub blob sidecar data'; - -CREATE TABLE tmp.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' AS tmp.libp2p_gossipsub_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_gossipsub_blob_sidecar_local, - unique_key -); - -INSERT INTO - tmp.libp2p_gossipsub_blob_sidecar -SELECT - toInt64( - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - wallclock_slot, - wallclock_slot_start_date_time, - wallclock_epoch, - wallclock_epoch_start_date_time, - propagation_slot_start_diff, - proposer_index, - blob_index, - parent_root, - state_root, - peer_id_unique_key, - message_id, - message_size, - topic_layer, - topic_fork_digest_value, - topic_name, - topic_encoding, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_gossipsub_blob_sidecar_local; - -DROP TABLE IF EXISTS default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_gossipsub_blob_sidecar_local -AND tmp.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' AS default.libp2p_gossipsub_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_gossipsub_blob_sidecar_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_handle_metadata -CREATE TABLE tmp.libp2p_handle_metadata_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the RPC', - `error` Nullable(String) COMMENT 'Error message if the metadata handling failed' CODEC(ZSTD(1)), - `protocol` LowCardinality(String) COMMENT 'The protocol of the metadata handling event', - `attnets` String COMMENT 'Attestation subnets the peer is subscribed to' CODEC(ZSTD(1)), - `seq_number` UInt64 COMMENT 'Sequence number of the metadata' CODEC(DoubleDelta, ZSTD(1)), - `syncnets` String COMMENT 'Sync subnets the peer is subscribed to' CODEC(ZSTD(1)), - `latency_milliseconds` Decimal(10, 3) COMMENT 'How long it took to handle the metadata request in milliseconds' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains the metadata handling events for libp2p peers.'; - -CREATE TABLE tmp.libp2p_handle_metadata ON CLUSTER '{cluster}' AS tmp.libp2p_handle_metadata_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_handle_metadata_local, - unique_key -); - -INSERT INTO - tmp.libp2p_handle_metadata -SELECT - toInt64( - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - seq_number - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - peer_id_unique_key, - error, - protocol, - attnets, - seq_number, - syncnets, - latency_milliseconds, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_handle_metadata_local; - -DROP TABLE IF EXISTS default.libp2p_handle_metadata ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_handle_metadata_local -AND tmp.libp2p_handle_metadata_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_handle_metadata ON CLUSTER '{cluster}' AS default.libp2p_handle_metadata_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_handle_metadata_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_handle_metadata ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_handle_metadata_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_handle_status -CREATE TABLE tmp.libp2p_handle_status_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `error` Nullable(String) COMMENT 'Error message if the status handling failed' CODEC(ZSTD(1)), - `protocol` LowCardinality(String) COMMENT 'The protocol of the status handling event', - `request_finalized_epoch` Nullable(UInt32) COMMENT 'Requested finalized epoch' CODEC(DoubleDelta, ZSTD(1)), - `request_finalized_root` Nullable(String) COMMENT 'Requested finalized root', - `request_fork_digest` LowCardinality(String) COMMENT 'Requested fork digest', - `request_head_root` Nullable(FixedString(66)) COMMENT 'Requested head root' CODEC(ZSTD(1)), - `request_head_slot` Nullable(UInt32) COMMENT 'Requested head slot' CODEC(ZSTD(1)), - `response_finalized_epoch` Nullable(UInt32) COMMENT 'Response finalized epoch' CODEC(DoubleDelta, ZSTD(1)), - `response_finalized_root` Nullable(FixedString(66)) COMMENT 'Response finalized root' CODEC(ZSTD(1)), - `response_fork_digest` LowCardinality(String) COMMENT 'Response fork digest', - `response_head_root` Nullable(FixedString(66)) COMMENT 'Response head root' CODEC(ZSTD(1)), - `response_head_slot` Nullable(UInt32) COMMENT 'Response head slot' CODEC(DoubleDelta, ZSTD(1)), - `latency_milliseconds` Decimal(10, 3) COMMENT 'How long it took to handle the status request in milliseconds' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(event_date_time) -ORDER BY - ( - event_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains the status handling events for libp2p peers.'; - -CREATE TABLE tmp.libp2p_handle_status ON CLUSTER '{cluster}' AS tmp.libp2p_handle_status_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_handle_status_local, - unique_key -); - -INSERT INTO - tmp.libp2p_handle_status -SELECT - toInt64( - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - peer_id_unique_key, - error, - protocol, - request_finalized_epoch, - request_finalized_root, - request_fork_digest, - request_head_root, - request_head_slot, - response_finalized_epoch, - response_finalized_root, - response_fork_digest, - response_head_root, - response_head_slot, - latency_milliseconds, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_handle_status_local; - -DROP TABLE IF EXISTS default.libp2p_handle_status ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_handle_status_local -AND tmp.libp2p_handle_status_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_handle_status ON CLUSTER '{cluster}' AS default.libp2p_handle_status_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_handle_status_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_handle_status ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_handle_status_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_join -CREATE TABLE tmp.libp2p_join_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer that joined the topic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains the details of the JOIN events from the libp2p client.'; - -CREATE TABLE tmp.libp2p_join ON CLUSTER '{cluster}' AS tmp.libp2p_join_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_join_local, - unique_key -); - -INSERT INTO - tmp.libp2p_join -SELECT - toInt64( - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - topic_layer, - topic_fork_digest_value, - topic_name, - topic_encoding, - peer_id_unique_key, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_join_local; - -DROP TABLE IF EXISTS default.libp2p_join ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_join_local -AND tmp.libp2p_join_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_join ON CLUSTER '{cluster}' AS default.libp2p_join_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_join_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_join ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_join_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_remove_peer -CREATE TABLE tmp.libp2p_remove_peer_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Contains the details of the peers removed from the libp2p client.'; - -CREATE TABLE tmp.libp2p_remove_peer ON CLUSTER '{cluster}' AS tmp.libp2p_remove_peer_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_remove_peer_local, - unique_key -); - -INSERT INTO - tmp.libp2p_remove_peer -SELECT - toInt64( - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - peer_id_unique_key, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_remove_peer_local; - -DROP TABLE IF EXISTS default.libp2p_remove_peer ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_remove_peer_local -AND tmp.libp2p_remove_peer_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_remove_peer ON CLUSTER '{cluster}' AS default.libp2p_remove_peer_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_remove_peer_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.libp2p_remove_peer ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_remove_peer_local ON CLUSTER '{cluster}' SYNC; - --- mempool_dumpster_transaction -CREATE TABLE tmp.mempool_dumpster_transaction_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'When this row was last updated, this is outside the source data and used for deduplication' CODEC(DoubleDelta, ZSTD(1)), - `timestamp` DateTime64(3) COMMENT 'Timestamp of the transaction' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), - `chain_id` UInt32 COMMENT 'The chain id of the transaction' CODEC(ZSTD(1)), - `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), - `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), - `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), - `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), - `gas_tip_cap` Nullable(UInt128) COMMENT 'The gas tip cap of the transaction in wei' CODEC(ZSTD(1)), - `gas_fee_cap` Nullable(UInt128) COMMENT 'The gas fee cap of the transaction in wei' CODEC(ZSTD(1)), - `data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `data_4bytes` Nullable(FixedString(10)) COMMENT 'The first 4 bytes of the call data of the transaction' CODEC(ZSTD(1)), - `sources` Array(LowCardinality(String)) COMMENT 'The sources that saw this transaction in their mempool', - `included_at_block_height` Nullable(UInt64) COMMENT 'The block height at which this transaction was included' CODEC(ZSTD(1)), - `included_block_timestamp` Nullable(DateTime64(3)) COMMENT 'The timestamp of the block at which this transaction was included' CODEC(DoubleDelta, ZSTD(1)), - `inclusion_delay_ms` Nullable(Int64) COMMENT 'The delay between the transaction timestamp and the block timestamp' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(timestamp) -ORDER BY - ( - timestamp, - unique_key, - chain_id - ) COMMENT 'Contains transactions from mempool dumpster dataset. Following the parquet schema with some additions'; - -CREATE TABLE tmp.mempool_dumpster_transaction ON CLUSTER '{cluster}' AS tmp.mempool_dumpster_transaction_local ENGINE = Distributed( - '{cluster}', - tmp, - mempool_dumpster_transaction_local, - unique_key -); - -INSERT INTO - tmp.mempool_dumpster_transaction -SELECT - toInt64( - cityHash64( - timestamp, - chain_id, - `from`, - nonce, - gas - ) - 9223372036854775808 - ), - NOW(), - timestamp, - hash, - chain_id, - `from`, - to, - value, - nonce, - gas, - gas_price, - gas_tip_cap, - gas_fee_cap, - data_size, - data_4bytes, - sources, - included_at_block_height, - included_block_timestamp, - inclusion_delay_ms -FROM - default.mempool_dumpster_transaction_local; - -DROP TABLE IF EXISTS default.mempool_dumpster_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.mempool_dumpster_transaction_local -AND tmp.mempool_dumpster_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.mempool_dumpster_transaction ON CLUSTER '{cluster}' AS default.mempool_dumpster_transaction_local ENGINE = Distributed( - '{cluster}', - default, - mempool_dumpster_transaction_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.mempool_dumpster_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.mempool_dumpster_transaction_local ON CLUSTER '{cluster}' SYNC; - --- mempool_transaction -CREATE TABLE tmp.mempool_transaction_local ON CLUSTER '{cluster}' ( - `unique_key` Int64 COMMENT 'Unique identifier for each record', - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'The time when the sentry saw the transaction in the mempool' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), - `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), - `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), - `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), - `gas_tip_cap` Nullable(UInt128) COMMENT 'The priority fee (tip) the user has set for the transaction', - `gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction', - `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), - `type` Nullable(UInt8) COMMENT 'The type of the transaction', - `size` UInt32 COMMENT 'The size of the transaction data in bytes' CODEC(ZSTD(1)), - `call_data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `blob_gas` Nullable(UInt64) COMMENT 'The maximum gas provided for the blob transaction execution', - `blob_gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction', - `blob_hashes` Array(String) COMMENT 'The hashes of the blob commitments for blob transactions', - `blob_sidecars_size` Nullable(UInt32) COMMENT 'The total size of the sidecars for blob transactions in bytes', - `blob_sidecars_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the sidecars for blob transactions in bytes', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_execution_fork_id_hash` LowCardinality(String) COMMENT 'The hash of the fork ID of the current Ethereum network', - `meta_execution_fork_id_next` LowCardinality(String) COMMENT 'The fork ID of the next planned Ethereum network upgrade', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(event_date_time) -ORDER BY - ( - event_date_time, - unique_key, - meta_network_name, - meta_client_name - ) COMMENT 'Each row represents a transaction that was seen in the mempool by a sentry client. Sentries can report the same transaction multiple times if it has been long enough since the last report.'; - -CREATE TABLE tmp.mempool_transaction ON CLUSTER '{cluster}' AS tmp.mempool_transaction_local ENGINE = Distributed( - '{cluster}', - tmp, - mempool_transaction_local, - unique_key -); - -INSERT INTO - tmp.mempool_transaction -SELECT - toInt64( - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - hash, - `from`, - nonce, - gas - ) - 9223372036854775808 - ), - NOW(), - event_date_time, - hash, - `from`, - to, - nonce, - gas_price, - gas, - gas_tip_cap, - gas_fee_cap, - value, - type, - size, - call_data_size, - blob_gas, - blob_gas_fee_cap, - blob_hashes, - blob_sidecars_size, - blob_sidecars_empty_size, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_execution_fork_id_hash, - meta_execution_fork_id_next, - meta_labels -FROM - default.mempool_transaction_local; - -DROP TABLE IF EXISTS default.mempool_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.mempool_transaction_local -AND tmp.mempool_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.mempool_transaction ON CLUSTER '{cluster}' AS default.mempool_transaction_local ENGINE = Distributed( - '{cluster}', - default, - mempool_transaction_local, - unique_key -); - -DROP TABLE IF EXISTS tmp.mempool_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.mempool_transaction_local ON CLUSTER '{cluster}' SYNC; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/041_remove_unique_field.up.sql b/deploy/migrations/clickhouse/041_remove_unique_field.up.sql deleted file mode 100644 index 62cff2a1..00000000 --- a/deploy/migrations/clickhouse/041_remove_unique_field.up.sql +++ /dev/null @@ -1,4780 +0,0 @@ --- beacon_api_eth_v1_beacon_committee -CREATE TABLE tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API committee payload', - `validators` Array(UInt32) COMMENT 'The validator indices in the beacon API committee payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - ) COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_beacon_committee_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_beacon_committee_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_beacon_committee -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - committee_index, - validators, - epoch, - epoch_start_date_time, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_beacon_committee_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_beacon_committee_local -AND tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_beacon_committee_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_beacon_committee_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_blob_sidecar -CREATE TABLE tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `blob_index` UInt64 COMMENT 'The index of blob sidecar in the beacon API event stream payload' CODEC(ZSTD(1)), - `kzg_commitment` FixedString(98) COMMENT 'The KZG commitment in the beacon API event stream payload' CODEC(ZSTD(1)), - `versioned_hash` FixedString(66) COMMENT 'The versioned hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - blob_index - ) COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_blob_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - blob_index - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_blob_sidecar -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - epoch, - epoch_start_date_time, - block_root, - blob_index, - kzg_commitment, - versioned_hash, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_blob_sidecar_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_blob_sidecar_local -AND tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_blob_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - blob_index - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_block -CREATE TABLE tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `block` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `execution_optimistic` Bool COMMENT 'If the attached beacon node is running in execution optimistic mode', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - block - ) COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_block_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_block_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_block -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - block, - epoch, - epoch_start_date_time, - execution_optimistic, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_block_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_block_local -AND tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_block_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_block_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_chain_reorg -CREATE TABLE tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `slot` UInt32 COMMENT 'The slot number of the chain reorg event in the beacon API event stream payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the reorg slot started', - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slots between when the reorg occurred and when the sentry received the event', - `depth` UInt16 COMMENT 'The depth of the chain reorg in the beacon API event stream payload', - `old_head_block` FixedString(66) COMMENT 'The old head block root hash in the beacon API event stream payload', - `new_head_block` FixedString(66) COMMENT 'The new head block root hash in the beacon API event stream payload', - `old_head_state` FixedString(66) COMMENT 'The old head state root hash in the beacon API event stream payload', - `new_head_state` FixedString(66) COMMENT 'The new head state root hash in the beacon API event stream payload', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `execution_optimistic` Bool COMMENT 'Whether the execution of the epoch was optimistic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - old_head_block, - new_head_block - ) COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_chain_reorg_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_chain_reorg_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - old_head_block, - new_head_block - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_chain_reorg -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - depth, - old_head_block, - new_head_block, - old_head_state, - new_head_state, - epoch, - epoch_start_date_time, - execution_optimistic, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_chain_reorg_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_chain_reorg_local -AND tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_chain_reorg_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_chain_reorg_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - old_head_block, - new_head_block - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_contribution_and_proof -CREATE TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `aggregator_index` UInt32 COMMENT 'The validator index of the aggregator in the beacon API event stream payload', - `contribution_slot` UInt32 COMMENT 'The slot number of the contribution in the beacon API event stream payload', - `contribution_slot_start_date_time` DateTime COMMENT 'The wall clock time when the contribution slot started', - `contribution_propagation_slot_start_diff` UInt32 COMMENT 'Difference in slots between when the contribution occurred and when the sentry received the event', - `contribution_beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload', - `contribution_subcommittee_index` LowCardinality(String) COMMENT 'The subcommittee index of the contribution in the beacon API event stream payload', - `contribution_aggregation_bits` String COMMENT 'The aggregation bits of the contribution in the beacon API event stream payload', - `contribution_signature` String COMMENT 'The signature of the contribution in the beacon API event stream payload', - `contribution_epoch` UInt32 COMMENT 'The epoch number of the contribution in the beacon API event stream payload', - `contribution_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the contribution epoch started', - `selection_proof` String COMMENT 'The selection proof in the beacon API event stream payload', - `signature` String COMMENT 'The signature in the beacon API event stream payload', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(contribution_slot_start_date_time) -ORDER BY - ( - contribution_slot_start_date_time, - meta_network_name, - meta_client_name, - contribution_beacon_block_root, - contribution_subcommittee_index, - signature - ) COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_contribution_and_proof_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_contribution_and_proof_local, - cityHash64( - contribution_slot_start_date_time, - meta_network_name, - meta_client_name, - contribution_beacon_block_root, - contribution_subcommittee_index, - signature - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_contribution_and_proof -SELECT - NOW(), - event_date_time, - aggregator_index, - contribution_slot, - contribution_slot_start_date_time, - contribution_propagation_slot_start_diff, - contribution_beacon_block_root, - contribution_subcommittee_index, - contribution_aggregation_bits, - contribution_signature, - contribution_epoch, - contribution_epoch_start_date_time, - selection_proof, - signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_contribution_and_proof_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_contribution_and_proof_local -AND tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_contribution_and_proof_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_contribution_and_proof_local, - cityHash64( - contribution_slot_start_date_time, - meta_network_name, - meta_client_name, - contribution_beacon_block_root, - contribution_subcommittee_index, - signature - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_finalized_checkpoint -CREATE TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `block` FixedString(66) COMMENT 'The finalized block root hash in the beacon API event stream payload', - `state` FixedString(66) COMMENT 'The finalized state root hash in the beacon API event stream payload', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `execution_optimistic` Bool COMMENT 'Whether the execution of the epoch was optimistic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY - ( - epoch_start_date_time, - meta_network_name, - meta_client_name, - block, - state - ) COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_finalized_checkpoint_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - meta_client_name, - block, - state - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_finalized_checkpoint -SELECT - NOW(), - event_date_time, - block, - state, - epoch, - epoch_start_date_time, - execution_optimistic, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_finalized_checkpoint_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_finalized_checkpoint_local -AND tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_finalized_checkpoint_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_finalized_checkpoint_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - meta_client_name, - block, - state - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_head -CREATE TABLE tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time', - `block` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `epoch_transition` Bool COMMENT 'If the event is an epoch transition', - `execution_optimistic` Bool COMMENT 'If the attached beacon node is running in execution optimistic mode', - `previous_duty_dependent_root` FixedString(66) COMMENT 'The previous duty dependent root in the beacon API event stream payload', - `current_duty_dependent_root` FixedString(66) COMMENT 'The current duty dependent root in the beacon API event stream payload', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - block, - previous_duty_dependent_root, - current_duty_dependent_root - ) COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_head_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_head_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block, - previous_duty_dependent_root, - current_duty_dependent_root - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_head -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - block, - epoch, - epoch_start_date_time, - epoch_transition, - execution_optimistic, - previous_duty_dependent_root, - current_duty_dependent_root, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_head_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_head_local -AND tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_head_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_head_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block, - previous_duty_dependent_root, - current_duty_dependent_root - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_events_voluntary_exit -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `validator_index` UInt32 COMMENT 'The index of the validator making the voluntary exit', - `signature` String COMMENT 'The signature of the voluntary exit in the beacon API event stream payload', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY - ( - epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_voluntary_exit_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_voluntary_exit -SELECT - NOW(), - event_date_time, - epoch, - epoch_start_date_time, - validator_index, - signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_voluntary_exit_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_voluntary_exit_local -AND tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_voluntary_exit_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_proposer_duty -CREATE TABLE tmp.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `proposer_validator_index` UInt32 COMMENT 'The validator index from the proposer duty payload' CODEC(ZSTD(1)), - `proposer_pubkey` String COMMENT 'The BLS public key of the validator from the proposer duty payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - proposer_validator_index - ) COMMENT 'Contains a proposer duty from a beacon block.'; - -CREATE TABLE tmp.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_proposer_duty_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_proposer_duty_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - proposer_validator_index - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_proposer_duty -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - proposer_validator_index, - proposer_pubkey, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_proposer_duty_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_proposer_duty_local -AND tmp.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_proposer_duty_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_proposer_duty_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - proposer_validator_index - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v1_validator_attestation_data -CREATE TABLE tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `slot` UInt32 COMMENT 'Slot number in the beacon API validator attestation data payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', - `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API validator attestation data payload', - `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API validator attestation data payload', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API validator attestation data payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `source_epoch` UInt32 COMMENT 'The source epoch number in the beacon API validator attestation data payload', - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started', - `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the beacon API validator attestation data payload', - `target_epoch` UInt32 COMMENT 'The target epoch number in the beacon API validator attestation data payload', - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started', - `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the beacon API validator attestation data payload', - `request_date_time` DateTime COMMENT 'When the request was sent to the beacon node', - `request_duration` UInt32 COMMENT 'The request duration in milliseconds', - `request_slot_start_diff` UInt32 COMMENT 'The difference between the request_date_time and the slot_start_date_time', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - beacon_block_root, - source_root, - target_root - ) COMMENT 'Contains beacon API validator attestation data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_validator_attestation_data_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_validator_attestation_data_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - beacon_block_root, - source_root, - target_root - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_validator_attestation_data -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - committee_index, - beacon_block_root, - epoch, - epoch_start_date_time, - source_epoch, - source_epoch_start_date_time, - source_root, - target_epoch, - target_epoch_start_date_time, - target_root, - request_date_time, - request_duration, - request_slot_start_diff, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_validator_attestation_data_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_validator_attestation_data_local -AND tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_validator_attestation_data_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_validator_attestation_data_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - beacon_block_root, - source_root, - target_root - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' SYNC; - --- beacon_api_eth_v2_beacon_block -CREATE TABLE tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry fetched the beacon block from a beacon node', - `slot` UInt32 COMMENT 'The slot number from beacon block payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the reorg slot started', - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block', - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `block_total_bytes` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload', - `block_total_bytes_compressed` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload when compressed using snappy', - `parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block', - `state_root` FixedString(66) COMMENT 'The root hash of the beacon state at this block', - `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block', - `eth1_data_block_hash` FixedString(66) COMMENT 'The block hash of the associated execution block', - `eth1_data_deposit_root` FixedString(66) COMMENT 'The root of the deposit tree in the associated execution block', - `execution_payload_block_hash` FixedString(66) COMMENT 'The block hash of the execution payload', - `execution_payload_block_number` UInt32 COMMENT 'The block number of the execution payload', - `execution_payload_fee_recipient` String COMMENT 'The recipient of the fee for this execution payload', - `execution_payload_state_root` FixedString(66) COMMENT 'The state root of the execution payload', - `execution_payload_parent_hash` FixedString(66) COMMENT 'The parent hash of the execution payload', - `execution_payload_transactions_count` Nullable(UInt32) COMMENT 'The transaction count of the execution payload', - `execution_payload_transactions_total_bytes` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload', - `execution_payload_transactions_total_bytes_compressed` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload when compressed using snappy', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - parent_root, - state_root - ) COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v2_beacon_block_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v2_beacon_block_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - parent_root, - state_root - ) -); - -INSERT INTO - tmp.beacon_api_eth_v2_beacon_block -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - block_total_bytes, - block_total_bytes_compressed, - parent_root, - state_root, - proposer_index, - eth1_data_block_hash, - eth1_data_deposit_root, - execution_payload_block_hash, - execution_payload_block_number, - execution_payload_fee_recipient, - execution_payload_state_root, - execution_payload_parent_hash, - execution_payload_transactions_count, - execution_payload_transactions_total_bytes, - execution_payload_transactions_total_bytes_compressed, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v2_beacon_block_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v2_beacon_block_local -AND tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v2_beacon_block_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v2_beacon_block_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - parent_root, - state_root - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' SYNC; - --- beacon_block_classification -CREATE TABLE tmp.beacon_block_classification_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block classification', - `slot` UInt32 COMMENT 'The slot number from beacon block classification', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block classification' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `best_guess_single` LowCardinality(String) COMMENT 'The best guess of the client that generated the beacon block', - `best_guess_multi` LowCardinality(String) COMMENT 'The best guess of the clients that generated the beacon block. This value will typically equal the best_guess_single value, but when multiple clients have high probabilities, this value will have multiple eg. "prysm or lighthouse"', - `client_probability_uncertain` Float32 COMMENT 'The probability that the client that generated the beacon block is uncertain' CODEC(ZSTD(1)), - `client_probability_prysm` Float32 COMMENT 'The probability that the client that generated the beacon block is Prysm' CODEC(ZSTD(1)), - `client_probability_teku` Float32 COMMENT 'The probability that the client that generated the beacon block is Teku' CODEC(ZSTD(1)), - `client_probability_nimbus` Float32 COMMENT 'The probability that the client that generated the beacon block is Nimbus' CODEC(ZSTD(1)), - `client_probability_lodestar` Float32 COMMENT 'The probability that the client that generated the beacon block is Lodestar' CODEC(ZSTD(1)), - `client_probability_grandine` Float32 COMMENT 'The probability that the client that generated the beacon block is Grandine' CODEC(ZSTD(1)), - `client_probability_lighthouse` Float32 COMMENT 'The probability that the client that generated the beacon block is Lighthouse' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - proposer_index - ) COMMENT 'Contains beacon block classification for a given slot. This is a best guess based on the client probabilities of the proposer. This is not guaranteed to be correct.'; - -CREATE TABLE tmp.beacon_block_classification ON CLUSTER '{cluster}' AS tmp.beacon_block_classification_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_block_classification_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - proposer_index - ) -); - -INSERT INTO - tmp.beacon_block_classification -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - best_guess_single, - best_guess_multi, - client_probability_uncertain, - client_probability_prysm, - client_probability_teku, - client_probability_nimbus, - client_probability_lodestar, - client_probability_grandine, - client_probability_lighthouse, - proposer_index, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_block_classification_local; - -DROP TABLE IF EXISTS default.beacon_block_classification ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_block_classification_local -AND tmp.beacon_block_classification_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_block_classification ON CLUSTER '{cluster}' AS default.beacon_block_classification_local ENGINE = Distributed( - '{cluster}', - default, - beacon_block_classification_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - proposer_index - ) -); - -DROP TABLE IF EXISTS tmp.beacon_block_classification ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_block_classification_local ON CLUSTER '{cluster}' SYNC; - --- beacon_p2p_attestation -CREATE TABLE tmp.beacon_p2p_attestation_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node', - `slot` UInt32 COMMENT 'Slot number in the beacon P2P payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon P2P payload', - `attesting_validator_index` Nullable(UInt32) COMMENT 'The index of the validator attesting to the event' CODEC(ZSTD(1)), - `attesting_validator_committee_index` LowCardinality(String) COMMENT 'The committee index of the attesting validator', - `aggregation_bits` String COMMENT 'The aggregation bits of the event in the beacon P2P payload' CODEC(ZSTD(1)), - `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch` UInt32 COMMENT 'The source epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `target_epoch` UInt32 COMMENT 'The target epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), - `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `attestation_subnet` LowCardinality(String) COMMENT 'The attestation subnet the attestation was gossiped on', - `validated` Bool COMMENT 'Whether the attestation was validated by the client', - `peer_id` String COMMENT 'The originating peer ID for the gossiped data' CODEC(ZSTD(1)), - `peer_latency` UInt32 COMMENT 'The latency of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_version` LowCardinality(String) COMMENT 'Peer client version that gossiped the data', - `peer_version_major` LowCardinality(String) COMMENT 'Peer client major version that gossiped the data', - `peer_version_minor` LowCardinality(String) COMMENT 'Peer client minor version that gossiped the data', - `peer_version_patch` LowCardinality(String) COMMENT 'Peer client patch version that gossiped the data', - `peer_implementation` LowCardinality(String) COMMENT 'Peer client implementation that gossiped the data', - `peer_ip` Nullable(IPv6) COMMENT 'IP address of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_city` LowCardinality(String) COMMENT 'City of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_country` LowCardinality(String) COMMENT 'Country of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_country_code` LowCardinality(String) COMMENT 'Country code of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the peer that gossiped the data' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - aggregation_bits, - beacon_block_root, - source_root, - target_root, - attestation_subnet, - peer_id - ) COMMENT 'Contains beacon chain P2P "attestation" data'; - -CREATE TABLE tmp.beacon_p2p_attestation ON CLUSTER '{cluster}' AS tmp.beacon_p2p_attestation_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_p2p_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - aggregation_bits, - beacon_block_root, - source_root, - target_root, - attestation_subnet, - peer_id - ) -); - -INSERT INTO - tmp.beacon_p2p_attestation -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - propagation_slot_start_diff, - committee_index, - attesting_validator_index, - attesting_validator_committee_index, - aggregation_bits, - beacon_block_root, - epoch, - epoch_start_date_time, - source_epoch, - source_epoch_start_date_time, - source_root, - target_epoch, - target_epoch_start_date_time, - target_root, - attestation_subnet, - validated, - peer_id, - peer_latency, - peer_version, - peer_version_major, - peer_version_minor, - peer_version_patch, - peer_implementation, - peer_ip, - peer_geo_city, - peer_geo_country, - peer_geo_country_code, - peer_geo_continent_code, - peer_geo_longitude, - peer_geo_latitude, - peer_geo_autonomous_system_number, - peer_geo_autonomous_system_organization, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_p2p_attestation_local; - -DROP TABLE IF EXISTS default.beacon_p2p_attestation ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_p2p_attestation_local -AND tmp.beacon_p2p_attestation_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_p2p_attestation ON CLUSTER '{cluster}' AS default.beacon_p2p_attestation_local ENGINE = Distributed( - '{cluster}', - default, - beacon_p2p_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - aggregation_bits, - beacon_block_root, - source_root, - target_root, - attestation_subnet, - peer_id - ) -); - -DROP TABLE IF EXISTS tmp.beacon_p2p_attestation ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_p2p_attestation_local ON CLUSTER '{cluster}' SYNC; - --- block_native_mempool_transaction -CREATE TABLE tmp.block_native_mempool_transaction_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `detecttime` DateTime64(3) COMMENT 'Timestamp that the transaction was detected in mempool' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'Unique identifier hash for a given transaction' CODEC(ZSTD(1)), - `status` LowCardinality(String) COMMENT 'Status of the transaction', - `region` LowCardinality(String) COMMENT 'The geographic region for the node that detected the transaction', - `reorg` Nullable(FixedString(66)) COMMENT 'If there was a reorg, refers to the blockhash of the reorg' CODEC(ZSTD(1)), - `replace` Nullable(FixedString(66)) COMMENT 'If the transaction was replaced (speedup/cancel), the transaction hash of the replacement' CODEC(ZSTD(1)), - `curblocknumber` Nullable(UInt64) COMMENT 'The block number the event was detected in' CODEC(ZSTD(1)), - `failurereason` Nullable(String) COMMENT 'If a transaction failed, this field provides contextual information' CODEC(ZSTD(1)), - `blockspending` Nullable(UInt64) COMMENT 'If a transaction was finalized (confirmed, failed), this refers to the number of blocks that the transaction was waiting to get on-chain' CODEC(ZSTD(1)), - `timepending` Nullable(UInt64) COMMENT 'If a transaction was finalized (confirmed, failed), this refers to the time in milliseconds that the transaction was waiting to get on-chain' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'A unique number which counts the number of transactions sent from a given address' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum number of gas units allowed for the transaction' CODEC(ZSTD(1)), - `gasprice` UInt128 COMMENT 'The price offered to the miner/validator per unit of gas. Denominated in wei' CODEC(ZSTD(1)), - `value` UInt128 COMMENT 'The amount of ETH transferred or sent to contract. Denominated in wei' CODEC(ZSTD(1)), - `toaddress` Nullable(FixedString(42)) COMMENT 'The destination of a given transaction' CODEC(ZSTD(1)), - `fromaddress` FixedString(42) COMMENT 'The source/initiator of a given transaction' CODEC(ZSTD(1)), - `datasize` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `data4bytes` Nullable(FixedString(10)) COMMENT 'The first 4 bytes of the call data of the transaction' CODEC(ZSTD(1)), - `network` LowCardinality(String) COMMENT 'The specific Ethereum network used', - `type` UInt8 COMMENT '"Post EIP-1559, this indicates how the gas parameters are submitted to the network: - type 0 - legacy - type 1 - usage of access lists according to EIP-2930 - type 2 - using maxpriorityfeepergas and maxfeepergas"' CODEC(ZSTD(1)), - `maxpriorityfeepergas` Nullable(UInt128) COMMENT 'The maximum value for a tip offered to the miner/validator per unit of gas. The actual tip paid can be lower if (maxfee - basefee) < maxpriorityfee. Denominated in wei' CODEC(ZSTD(1)), - `maxfeepergas` Nullable(UInt128) COMMENT 'The maximum value for the transaction fee (including basefee and tip) offered to the miner/validator per unit of gas. Denominated in wei' CODEC(ZSTD(1)), - `basefeepergas` Nullable(UInt128) COMMENT 'The fee per unit of gas paid and burned for the curblocknumber. This fee is algorithmically determined. Denominated in wei' CODEC(ZSTD(1)), - `dropreason` Nullable(String) COMMENT 'If the transaction was dropped from the mempool, this describes the contextual reason for the drop' CODEC(ZSTD(1)), - `rejectionreason` Nullable(String) COMMENT 'If the transaction was rejected from the mempool, this describes the contextual reason for the rejection' CODEC(ZSTD(1)), - `stuck` Bool COMMENT 'A transaction was detected in the queued area of the mempool and is not eligible for inclusion in a block' CODEC(ZSTD(1)), - `gasused` Nullable(UInt64) COMMENT 'If the transaction was published on-chain, this value indicates the amount of gas that was actually consumed. Denominated in wei' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(detecttime) -ORDER BY - ( - detecttime, - network, - hash, - fromaddress, - nonce, - gas - ) COMMENT 'Contains transactions from block native mempool dataset'; - -CREATE TABLE tmp.block_native_mempool_transaction ON CLUSTER '{cluster}' AS tmp.block_native_mempool_transaction_local ENGINE = Distributed( - '{cluster}', - tmp, - block_native_mempool_transaction_local, - cityHash64( - detecttime, - network, - hash, - fromaddress, - nonce, - gas - ) -); - -INSERT INTO - tmp.block_native_mempool_transaction -SELECT - NOW(), - detecttime, - hash, - status, - region, - reorg, - replace, - curblocknumber, - failurereason, - blockspending, - timepending, - nonce, - gas, - gasprice, - value, - toaddress, - fromaddress, - datasize, - data4bytes, - network, - type, - maxpriorityfeepergas, - maxfeepergas, - basefeepergas, - dropreason, - rejectionreason, - stuck, - gasused -FROM - default.block_native_mempool_transaction_local; - -DROP TABLE IF EXISTS default.block_native_mempool_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.block_native_mempool_transaction_local -AND tmp.block_native_mempool_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.block_native_mempool_transaction ON CLUSTER '{cluster}' AS default.block_native_mempool_transaction_local ENGINE = Distributed( - '{cluster}', - default, - block_native_mempool_transaction_local, - cityHash64( - detecttime, - network, - hash, - fromaddress, - nonce, - gas - ) -); - -DROP TABLE IF EXISTS tmp.block_native_mempool_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.block_native_mempool_transaction_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_blob_sidecar -CREATE TABLE tmp.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block' CODEC(ZSTD(1)), - `versioned_hash` FixedString(66) COMMENT 'The versioned hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `kzg_commitment` FixedString(98) COMMENT 'The KZG commitment in the blob sidecar payload' CODEC(ZSTD(1)), - `kzg_proof` FixedString(98) COMMENT 'The KZG proof in the blob sidecar payload' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), - `blob_index` UInt64 COMMENT 'The index of blob sidecar in the blob sidecar payload' CODEC(ZSTD(1)), - `blob_size` UInt32 COMMENT 'The total bytes of the blob' CODEC(ZSTD(1)), - `blob_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the blob in bytes' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - blob_index - ) COMMENT 'Contains a blob sidecar from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' AS tmp.canonical_beacon_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_blob_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - blob_index - ) -); - -INSERT INTO - tmp.canonical_beacon_blob_sidecar -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_parent_root, - versioned_hash, - kzg_commitment, - kzg_proof, - proposer_index, - blob_index, - blob_size, - blob_empty_size, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_blob_sidecar_local; - -DROP TABLE IF EXISTS default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_blob_sidecar_local -AND tmp.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' AS default.canonical_beacon_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_blob_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - blob_index - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_attester_slashing -CREATE TABLE tmp.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `attestation_1_attesting_indices` Array(UInt32) COMMENT 'The attesting indices from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_signature` String COMMENT 'The signature from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_data_beacon_block_root` FixedString(66) COMMENT 'The beacon block root from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_data_slot` UInt32 COMMENT 'The slot number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_1_data_index` UInt32 COMMENT 'The attestor index from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_data_source_epoch` UInt32 COMMENT 'The source epoch number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_1_data_source_root` FixedString(66) COMMENT 'The source root from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_1_data_target_epoch` UInt32 COMMENT 'The target epoch number from the first attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_1_data_target_root` FixedString(66) COMMENT 'The target root from the first attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_attesting_indices` Array(UInt32) COMMENT 'The attesting indices from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_signature` String COMMENT 'The signature from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_data_beacon_block_root` FixedString(66) COMMENT 'The beacon block root from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_data_slot` UInt32 COMMENT 'The slot number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_2_data_index` UInt32 COMMENT 'The attestor index from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_data_source_epoch` UInt32 COMMENT 'The source epoch number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_2_data_source_root` FixedString(66) COMMENT 'The source root from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `attestation_2_data_target_epoch` UInt32 COMMENT 'The target epoch number from the second attestation in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `attestation_2_data_target_root` FixedString(66) COMMENT 'The target root from the second attestation in the slashing payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - attestation_1_attesting_indices, - attestation_2_attesting_indices, - attestation_1_data_slot, - attestation_2_data_slot, - attestation_1_data_beacon_block_root, - attestation_2_data_beacon_block_root - ) COMMENT 'Contains attester slashing from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_attester_slashing_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_attester_slashing_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - attestation_1_attesting_indices, - attestation_2_attesting_indices, - attestation_1_data_slot, - attestation_2_data_slot, - attestation_1_data_beacon_block_root, - attestation_2_data_beacon_block_root - ) -); - -INSERT INTO - tmp.canonical_beacon_block_attester_slashing -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - attestation_1_attesting_indices, - attestation_1_signature, - attestation_1_data_beacon_block_root, - attestation_1_data_slot, - attestation_1_data_index, - attestation_1_data_source_epoch, - attestation_1_data_source_root, - attestation_1_data_target_epoch, - attestation_1_data_target_root, - attestation_2_attesting_indices, - attestation_2_signature, - attestation_2_data_beacon_block_root, - attestation_2_data_slot, - attestation_2_data_index, - attestation_2_data_source_epoch, - attestation_2_data_source_root, - attestation_2_data_target_epoch, - attestation_2_data_target_root, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_attester_slashing_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_attester_slashing_local -AND tmp.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' AS default.canonical_beacon_block_attester_slashing_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_attester_slashing_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - attestation_1_attesting_indices, - attestation_2_attesting_indices, - attestation_1_data_slot, - attestation_2_data_slot, - attestation_1_data_beacon_block_root, - attestation_2_data_beacon_block_root - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_bls_to_execution_change -CREATE TABLE tmp.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `exchanging_message_validator_index` UInt32 COMMENT 'The validator index from the exchanging message' CODEC(ZSTD(1)), - `exchanging_message_from_bls_pubkey` String COMMENT 'The BLS public key from the exchanging message' CODEC(ZSTD(1)), - `exchanging_message_to_execution_address` FixedString(42) COMMENT 'The execution address from the exchanging message' CODEC(ZSTD(1)), - `exchanging_signature` String COMMENT 'The signature for the exchanging message' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - exchanging_message_validator_index, - exchanging_message_from_bls_pubkey, - exchanging_message_to_execution_address - ) COMMENT 'Contains bls to execution change from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_bls_to_execution_change_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_bls_to_execution_change_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - exchanging_message_validator_index, - exchanging_message_from_bls_pubkey, - exchanging_message_to_execution_address - ) -); - -INSERT INTO - tmp.canonical_beacon_block_bls_to_execution_change -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - exchanging_message_validator_index, - exchanging_message_from_bls_pubkey, - exchanging_message_to_execution_address, - exchanging_signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_bls_to_execution_change_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_bls_to_execution_change_local -AND tmp.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' AS default.canonical_beacon_block_bls_to_execution_change_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_bls_to_execution_change_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - exchanging_message_validator_index, - exchanging_message_from_bls_pubkey, - exchanging_message_to_execution_address - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_deposit -CREATE TABLE tmp.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `deposit_proof` Array(String) COMMENT 'The proof of the deposit data' CODEC(ZSTD(1)), - `deposit_data_pubkey` String COMMENT 'The BLS public key of the validator from the deposit data' CODEC(ZSTD(1)), - `deposit_data_withdrawal_credentials` FixedString(66) COMMENT 'The withdrawal credentials of the validator from the deposit data' CODEC(ZSTD(1)), - `deposit_data_amount` UInt128 COMMENT 'The amount of the deposit from the deposit data' CODEC(ZSTD(1)), - `deposit_data_signature` String COMMENT 'The signature of the deposit data' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - deposit_data_pubkey, - deposit_proof - ) COMMENT 'Contains a deposit from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_deposit ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_deposit_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_deposit_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - deposit_data_pubkey, - deposit_proof - ) -); - -INSERT INTO - tmp.canonical_beacon_block_deposit -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - deposit_proof, - deposit_data_pubkey, - deposit_data_withdrawal_credentials, - deposit_data_amount, - deposit_data_signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_deposit_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_deposit_local -AND tmp.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' AS default.canonical_beacon_block_deposit_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_deposit_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - deposit_data_pubkey, - deposit_proof - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_deposit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_execution_transaction -CREATE TABLE tmp.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `position` UInt32 COMMENT 'The position of the transaction in the beacon block' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), - `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), - `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), - `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), - `gas_tip_cap` Nullable(UInt128) COMMENT 'The priority fee (tip) the user has set for the transaction' CODEC(ZSTD(1)), - `gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), - `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), - `type` UInt8 COMMENT 'The type of the transaction' CODEC(ZSTD(1)), - `size` UInt32 COMMENT 'The size of the transaction data in bytes' CODEC(ZSTD(1)), - `call_data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `blob_gas` Nullable(UInt64) COMMENT 'The maximum gas provided for the blob transaction execution' CODEC(ZSTD(1)), - `blob_gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), - `blob_hashes` Array(String) COMMENT 'The hashes of the blob commitments for blob transactions' CODEC(ZSTD(1)), - `blob_sidecars_size` Nullable(UInt32) COMMENT 'The total size of the sidecars for blob transactions in bytes' CODEC(ZSTD(1)), - `blob_sidecars_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the sidecars for blob transactions in bytes' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - position, - hash, - nonce - ) COMMENT 'Contains execution transaction from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_execution_transaction_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_execution_transaction_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - position, - hash, - nonce - ) -); - -INSERT INTO - tmp.canonical_beacon_block_execution_transaction -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - position, - hash, - `from`, - to, - nonce, - gas_price, - gas, - gas_tip_cap, - gas_fee_cap, - value, - type, - size, - call_data_size, - blob_gas, - blob_gas_fee_cap, - blob_hashes, - blob_sidecars_size, - blob_sidecars_empty_size, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_execution_transaction_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_execution_transaction_local -AND tmp.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' AS default.canonical_beacon_block_execution_transaction_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_execution_transaction_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - position, - hash, - nonce - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block -CREATE TABLE tmp.canonical_beacon_block_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `block_total_bytes` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload' CODEC(ZSTD(1)), - `block_total_bytes_compressed` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload when compressed using snappy' CODEC(ZSTD(1)), - `parent_root` FixedString(66) COMMENT 'The root hash of the parent beacon block' CODEC(ZSTD(1)), - `state_root` FixedString(66) COMMENT 'The root hash of the beacon state at this block' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The index of the validator that proposed the beacon block' CODEC(ZSTD(1)), - `eth1_data_block_hash` FixedString(66) COMMENT 'The block hash of the associated execution block' CODEC(ZSTD(1)), - `eth1_data_deposit_root` FixedString(66) COMMENT 'The root of the deposit tree in the associated execution block' CODEC(ZSTD(1)), - `execution_payload_block_hash` FixedString(66) COMMENT 'The block hash of the execution payload' CODEC(ZSTD(1)), - `execution_payload_block_number` UInt32 COMMENT 'The block number of the execution payload' CODEC(DoubleDelta, ZSTD(1)), - `execution_payload_fee_recipient` String COMMENT 'The recipient of the fee for this execution payload' CODEC(ZSTD(1)), - `execution_payload_state_root` FixedString(66) COMMENT 'The state root of the execution payload' CODEC(ZSTD(1)), - `execution_payload_parent_hash` FixedString(66) COMMENT 'The parent hash of the execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_count` Nullable(UInt32) COMMENT 'The transaction count of the execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_total_bytes` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_total_bytes_compressed` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload when compressed using snappy' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - (slot_start_date_time, meta_network_name) COMMENT 'Contains beacon block from a beacon node.'; - -CREATE TABLE tmp.canonical_beacon_block ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_local, - cityHash64( - slot_start_date_time, - meta_network_name - ) -); - -INSERT INTO - tmp.canonical_beacon_block -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - block_total_bytes, - block_total_bytes_compressed, - parent_root, - state_root, - proposer_index, - eth1_data_block_hash, - eth1_data_deposit_root, - execution_payload_block_hash, - execution_payload_block_number, - execution_payload_fee_recipient, - execution_payload_state_root, - execution_payload_parent_hash, - execution_payload_transactions_count, - execution_payload_transactions_total_bytes, - execution_payload_transactions_total_bytes_compressed, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_local -AND tmp.canonical_beacon_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block ON CLUSTER '{cluster}' AS default.canonical_beacon_block_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_local, - cityHash64( - slot_start_date_time, - meta_network_name - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_proposer_slashing -CREATE TABLE tmp.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `signed_header_1_message_slot` UInt32 COMMENT 'The slot number from the first signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `signed_header_1_message_proposer_index` UInt32 COMMENT 'The proposer index from the first signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `signed_header_1_message_body_root` FixedString(66) COMMENT 'The body root from the first signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_1_message_parent_root` FixedString(66) COMMENT 'The parent root from the first signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_1_message_state_root` FixedString(66) COMMENT 'The state root from the first signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_1_signature` String COMMENT 'The signature for the first signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_2_message_slot` UInt32 COMMENT 'The slot number from the second signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `signed_header_2_message_proposer_index` UInt32 COMMENT 'The proposer index from the second signed header in the slashing payload' CODEC(DoubleDelta, ZSTD(1)), - `signed_header_2_message_body_root` FixedString(66) COMMENT 'The body root from the second signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_2_message_parent_root` FixedString(66) COMMENT 'The parent root from the second signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_2_message_state_root` FixedString(66) COMMENT 'The state root from the second signed header in the slashing payload' CODEC(ZSTD(1)), - `signed_header_2_signature` String COMMENT 'The signature for the second signed header in the slashing payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - signed_header_1_message_slot, - signed_header_2_message_slot, - signed_header_1_message_proposer_index, - signed_header_2_message_proposer_index, - signed_header_1_message_body_root, - signed_header_2_message_body_root - ) COMMENT 'Contains proposer slashing from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_proposer_slashing_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_proposer_slashing_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - signed_header_1_message_slot, - signed_header_2_message_slot, - signed_header_1_message_proposer_index, - signed_header_2_message_proposer_index, - signed_header_1_message_body_root, - signed_header_2_message_body_root - ) -); - -INSERT INTO - tmp.canonical_beacon_block_proposer_slashing -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - signed_header_1_message_slot, - signed_header_1_message_proposer_index, - signed_header_1_message_body_root, - signed_header_1_message_parent_root, - signed_header_1_message_state_root, - signed_header_1_signature, - signed_header_2_message_slot, - signed_header_2_message_proposer_index, - signed_header_2_message_body_root, - signed_header_2_message_parent_root, - signed_header_2_message_state_root, - signed_header_2_signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_proposer_slashing_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_proposer_slashing_local -AND tmp.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' AS default.canonical_beacon_block_proposer_slashing_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_proposer_slashing_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - signed_header_1_message_slot, - signed_header_2_message_slot, - signed_header_1_message_proposer_index, - signed_header_2_message_proposer_index, - signed_header_1_message_body_root, - signed_header_2_message_body_root - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_voluntary_exit -CREATE TABLE tmp.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `voluntary_exit_message_epoch` UInt32 COMMENT 'The epoch number from the exit message' CODEC(DoubleDelta, ZSTD(1)), - `voluntary_exit_message_validator_index` UInt32 COMMENT 'The validator index from the exit message' CODEC(ZSTD(1)), - `voluntary_exit_signature` String COMMENT 'The signature of the exit message' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - voluntary_exit_message_epoch, - voluntary_exit_message_validator_index - ) COMMENT 'Contains a voluntary exit from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_voluntary_exit_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - voluntary_exit_message_epoch, - voluntary_exit_message_validator_index - ) -); - -INSERT INTO - tmp.canonical_beacon_block_voluntary_exit -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - voluntary_exit_message_epoch, - voluntary_exit_message_validator_index, - voluntary_exit_signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_voluntary_exit_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_voluntary_exit_local -AND tmp.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' AS default.canonical_beacon_block_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_voluntary_exit_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - voluntary_exit_message_epoch, - voluntary_exit_message_validator_index - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_block_withdrawal -CREATE TABLE tmp.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `withdrawal_index` UInt32 COMMENT 'The index of the withdrawal' CODEC(ZSTD(1)), - `withdrawal_validator_index` UInt32 COMMENT 'The validator index from the withdrawal data' CODEC(ZSTD(1)), - `withdrawal_address` FixedString(42) COMMENT 'The address of the account that is the withdrawal recipient' CODEC(ZSTD(1)), - `withdrawal_amount` UInt128 COMMENT 'The amount of the withdrawal from the withdrawal data' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - withdrawal_index, - withdrawal_validator_index - ) COMMENT 'Contains a withdrawal from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' AS tmp.canonical_beacon_block_withdrawal_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_block_withdrawal_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - withdrawal_index, - withdrawal_validator_index - ) -); - -INSERT INTO - tmp.canonical_beacon_block_withdrawal -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - block_root, - block_version, - withdrawal_index, - withdrawal_validator_index, - withdrawal_address, - withdrawal_amount, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_block_withdrawal_local; - -DROP TABLE IF EXISTS default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_block_withdrawal_local -AND tmp.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' AS default.canonical_beacon_block_withdrawal_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_withdrawal_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - withdrawal_index, - withdrawal_validator_index - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_elaborated_attestation -CREATE TABLE tmp.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the elaborated attestation from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `block_slot` UInt32 COMMENT 'The slot number of the block containing the attestation' CODEC(DoubleDelta, ZSTD(1)), - `block_slot_start_date_time` DateTime COMMENT 'The wall clock time when the block slot started' CODEC(DoubleDelta, ZSTD(1)), - `block_epoch` UInt32 COMMENT 'The epoch number of the block containing the attestation' CODEC(DoubleDelta, ZSTD(1)), - `block_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the block epoch started' CODEC(DoubleDelta, ZSTD(1)), - `position_in_block` UInt32 COMMENT 'The position of the attestation in the block' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root of the block containing the attestation' CODEC(ZSTD(1)), - `validators` Array(UInt32) COMMENT 'Array of validator indices participating in the attestation' CODEC(ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The index of the committee making the attestation', - `beacon_block_root` FixedString(66) COMMENT 'The root of the beacon block being attested to' CODEC(ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number being attested to' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), - `source_epoch` UInt32 COMMENT 'The source epoch referenced in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_root` FixedString(66) COMMENT 'The root of the source checkpoint in the attestation' CODEC(ZSTD(1)), - `target_epoch` UInt32 COMMENT 'The target epoch referenced in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), - `target_root` FixedString(66) COMMENT 'The root of the target checkpoint in the attestation' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - block_root, - block_slot, - position_in_block, - beacon_block_root, - slot, - committee_index, - source_root, - target_root - ) COMMENT 'Contains elaborated attestations from beacon blocks.'; - -CREATE TABLE tmp.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' AS tmp.canonical_beacon_elaborated_attestation_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_elaborated_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - block_slot, - position_in_block, - beacon_block_root, - slot, - committee_index, - source_root, - target_root - ) -); - -INSERT INTO - tmp.canonical_beacon_elaborated_attestation -SELECT - NOW(), - event_date_time, - block_slot, - block_slot_start_date_time, - block_epoch, - block_epoch_start_date_time, - position_in_block, - block_root, - validators, - committee_index, - beacon_block_root, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - source_epoch, - source_epoch_start_date_time, - source_root, - target_epoch, - target_epoch_start_date_time, - target_root, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_elaborated_attestation_local; - -DROP TABLE IF EXISTS default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_elaborated_attestation_local -AND tmp.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' AS default.canonical_beacon_elaborated_attestation_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_elaborated_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - block_slot, - position_in_block, - beacon_block_root, - slot, - committee_index, - source_root, - target_root - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' SYNC; - --- canonical_beacon_proposer_duty -CREATE TABLE tmp.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the proposer duty information from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'The slot number for which the proposer duty is assigned' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number containing the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `proposer_validator_index` UInt32 COMMENT 'The validator index of the proposer for the slot' CODEC(ZSTD(1)), - `proposer_pubkey` String COMMENT 'The public key of the validator proposer' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the even' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - proposer_validator_index, - proposer_pubkey - ) COMMENT 'Contains a proposer duty from a beacon block.'; - -CREATE TABLE tmp.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' AS tmp.canonical_beacon_proposer_duty_local ENGINE = Distributed( - '{cluster}', - tmp, - canonical_beacon_proposer_duty_local, - cityHash64( - slot_start_date_time, - meta_network_name, - proposer_validator_index, - proposer_pubkey - ) -); - -INSERT INTO - tmp.canonical_beacon_proposer_duty -SELECT - NOW(), - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - proposer_validator_index, - proposer_pubkey, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.canonical_beacon_proposer_duty_local; - -DROP TABLE IF EXISTS default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.canonical_beacon_proposer_duty_local -AND tmp.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' AS default.canonical_beacon_proposer_duty_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_proposer_duty_local, - cityHash64( - slot_start_date_time, - meta_network_name, - proposer_validator_index, - proposer_pubkey - ) -); - -DROP TABLE IF EXISTS tmp.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_add_peer -CREATE TABLE tmp.libp2p_add_peer_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `protocol` LowCardinality(String) COMMENT 'Protocol used by the peer', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) COMMENT 'Contains the details of the peers added to the libp2p client.'; - -CREATE TABLE tmp.libp2p_add_peer ON CLUSTER '{cluster}' AS tmp.libp2p_add_peer_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_add_peer_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) -); - -INSERT INTO - tmp.libp2p_add_peer -SELECT - NOW(), - event_date_time, - peer_id_unique_key, - protocol, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_add_peer_local; - -DROP TABLE IF EXISTS default.libp2p_add_peer ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_add_peer_local -AND tmp.libp2p_add_peer_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_add_peer ON CLUSTER '{cluster}' AS default.libp2p_add_peer_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_add_peer_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_add_peer ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_add_peer_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_connected -CREATE TABLE tmp.libp2p_connected_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `remote_peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the remote peer', - `remote_protocol` LowCardinality(String) COMMENT 'Protocol of the remote peer', - `remote_transport_protocol` LowCardinality(String) COMMENT 'Transport protocol of the remote peer', - `remote_port` UInt16 COMMENT 'Port of the remote peer' CODEC(ZSTD(1)), - `remote_ip` Nullable(IPv6) COMMENT 'IP address of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_city` LowCardinality(String) COMMENT 'City of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_country` LowCardinality(String) COMMENT 'Country of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_country_code` LowCardinality(String) COMMENT 'Country code of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_agent_implementation` LowCardinality(String) COMMENT 'Implementation of the remote peer', - `remote_agent_version` LowCardinality(String) COMMENT 'Version of the remote peer', - `remote_agent_version_major` LowCardinality(String) COMMENT 'Major version of the remote peer', - `remote_agent_version_minor` LowCardinality(String) COMMENT 'Minor version of the remote peer', - `remote_agent_version_patch` LowCardinality(String) COMMENT 'Patch version of the remote peer', - `remote_agent_platform` LowCardinality(String) COMMENT 'Platform of the remote peer', - `direction` LowCardinality(String) COMMENT 'Connection direction', - `opened` DateTime COMMENT 'Timestamp when the connection was opened' CODEC(DoubleDelta, ZSTD(1)), - `transient` Bool COMMENT 'Whether the connection is transient', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - direction, - opened - ) COMMENT 'Contains the details of the CONNECTED events from the libp2p client.'; - -CREATE TABLE tmp.libp2p_connected ON CLUSTER '{cluster}' AS tmp.libp2p_connected_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_connected_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - direction, - opened - ) -); - -INSERT INTO - tmp.libp2p_connected -SELECT - NOW(), - event_date_time, - remote_peer_id_unique_key, - remote_protocol, - remote_transport_protocol, - remote_port, - remote_ip, - remote_geo_city, - remote_geo_country, - remote_geo_country_code, - remote_geo_continent_code, - remote_geo_longitude, - remote_geo_latitude, - remote_geo_autonomous_system_number, - remote_geo_autonomous_system_organization, - remote_agent_implementation, - remote_agent_version, - remote_agent_version_major, - remote_agent_version_minor, - remote_agent_version_patch, - remote_agent_platform, - direction, - opened, - transient, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_connected_local; - -DROP TABLE IF EXISTS default.libp2p_connected ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_connected_local -AND tmp.libp2p_connected_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_connected ON CLUSTER '{cluster}' AS default.libp2p_connected_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_connected_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - direction, - opened - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_connected ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_connected_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_disconnected -CREATE TABLE tmp.libp2p_disconnected_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `remote_peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the remote peer', - `remote_protocol` LowCardinality(String) COMMENT 'Protocol of the remote peer', - `remote_transport_protocol` LowCardinality(String) COMMENT 'Transport protocol of the remote peer', - `remote_port` UInt16 COMMENT 'Port of the remote peer' CODEC(ZSTD(1)), - `remote_ip` Nullable(IPv6) COMMENT 'IP address of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_city` LowCardinality(String) COMMENT 'City of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_country` LowCardinality(String) COMMENT 'Country of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_country_code` LowCardinality(String) COMMENT 'Country code of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the remote peer that generated the event' CODEC(ZSTD(1)), - `remote_agent_implementation` LowCardinality(String) COMMENT 'Implementation of the remote peer', - `remote_agent_version` LowCardinality(String) COMMENT 'Version of the remote peer', - `remote_agent_version_major` LowCardinality(String) COMMENT 'Major version of the remote peer', - `remote_agent_version_minor` LowCardinality(String) COMMENT 'Minor version of the remote peer', - `remote_agent_version_patch` LowCardinality(String) COMMENT 'Patch version of the remote peer', - `remote_agent_platform` LowCardinality(String) COMMENT 'Platform of the remote peer', - `direction` LowCardinality(String) COMMENT 'Connection direction', - `opened` DateTime COMMENT 'Timestamp when the connection was opened' CODEC(DoubleDelta, ZSTD(1)), - `transient` Bool COMMENT 'Whether the connection is transient', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - direction, - opened - ) COMMENT 'Contains the details of the DISCONNECTED events from the libp2p client.'; - -CREATE TABLE tmp.libp2p_disconnected ON CLUSTER '{cluster}' AS tmp.libp2p_disconnected_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_disconnected_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - direction, - opened - ) -); - -INSERT INTO - tmp.libp2p_disconnected -SELECT - NOW(), - event_date_time, - remote_peer_id_unique_key, - remote_protocol, - remote_transport_protocol, - remote_port, - remote_ip, - remote_geo_city, - remote_geo_country, - remote_geo_country_code, - remote_geo_continent_code, - remote_geo_longitude, - remote_geo_latitude, - remote_geo_autonomous_system_number, - remote_geo_autonomous_system_organization, - remote_agent_implementation, - remote_agent_version, - remote_agent_version_major, - remote_agent_version_minor, - remote_agent_version_patch, - remote_agent_platform, - direction, - opened, - transient, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_disconnected_local; - -DROP TABLE IF EXISTS default.libp2p_disconnected ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_disconnected_local -AND tmp.libp2p_disconnected_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_disconnected ON CLUSTER '{cluster}' AS default.libp2p_disconnected_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_disconnected_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - direction, - opened - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_disconnected ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_disconnected_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_gossipsub_beacon_attestation -CREATE TABLE tmp.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - -- ensure the first time this attestation was seen by a peer is in this table - -- 4294967295 = UInt32 max - `version` UInt32 DEFAULT 4294967295 - propagation_slot_start_diff COMMENT 'Version of this row, to help with de-duplication we want the latest updated_date_time but lowest propagation_slot_start_diff time' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The committee index in the attestation', - `attesting_validator_index` Nullable(UInt32) COMMENT 'The index of the validator attesting to the event' CODEC(ZSTD(1)), - `attesting_validator_committee_index` LowCardinality(String) COMMENT 'The committee index of the attesting validator', - `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', - `aggregation_bits` String COMMENT 'The aggregation bits of the event in the attestation' CODEC(ZSTD(1)), - `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the attestation' CODEC(ZSTD(1)), - `source_epoch` UInt32 COMMENT 'The source epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the attestation' CODEC(ZSTD(1)), - `target_epoch` UInt32 COMMENT 'The target epoch number in the attestation' CODEC(DoubleDelta, ZSTD(1)), - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), - `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the attestation' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Network ID associated with the client' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - `version` -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) COMMENT 'Table for libp2p gossipsub beacon attestation data.'; - -CREATE TABLE tmp.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' AS tmp.libp2p_gossipsub_beacon_attestation_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_gossipsub_beacon_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) -); - -INSERT INTO - tmp.libp2p_gossipsub_beacon_attestation -SELECT - NOW(), - NULL, - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - committee_index, - attesting_validator_index, - attesting_validator_committee_index, - wallclock_slot, - wallclock_slot_start_date_time, - wallclock_epoch, - wallclock_epoch_start_date_time, - propagation_slot_start_diff, - peer_id_unique_key, - message_id, - message_size, - topic_layer, - topic_fork_digest_value, - topic_name, - topic_encoding, - aggregation_bits, - beacon_block_root, - source_epoch, - source_epoch_start_date_time, - source_root, - target_epoch, - target_epoch_start_date_time, - target_root, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_gossipsub_beacon_attestation_local; - -DROP TABLE IF EXISTS default.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_gossipsub_beacon_attestation_local -AND tmp.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' AS default.libp2p_gossipsub_beacon_attestation_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_gossipsub_beacon_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_gossipsub_beacon_block -CREATE TABLE tmp.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - -- ensure the first time this block was seen by a peer is in this table - -- 4294967295 = UInt32 max - `version` UInt32 DEFAULT 4294967295 - propagation_slot_start_diff COMMENT 'Version of this row, to help with de-duplication we want the latest updated_date_time but lowest propagation_slot_start_diff time' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), - `block` FixedString(66) COMMENT 'The beacon block root hash' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The proposer index of the beacon block' CODEC(ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Network ID associated with the client' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - `version` -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) COMMENT 'Table for libp2p gossipsub beacon block data.'; - -CREATE TABLE tmp.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' AS tmp.libp2p_gossipsub_beacon_block_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_gossipsub_beacon_block_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) -); - -INSERT INTO - tmp.libp2p_gossipsub_beacon_block -SELECT - NOW(), - NULL, - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - wallclock_slot, - wallclock_slot_start_date_time, - wallclock_epoch, - wallclock_epoch_start_date_time, - propagation_slot_start_diff, - block, - proposer_index, - peer_id_unique_key, - message_id, - message_size, - topic_layer, - topic_fork_digest_value, - topic_name, - topic_encoding, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_gossipsub_beacon_block_local; - -DROP TABLE IF EXISTS default.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_gossipsub_beacon_block_local -AND tmp.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' AS default.libp2p_gossipsub_beacon_block_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_gossipsub_beacon_block_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_gossipsub_blob_sidecar -CREATE TABLE tmp.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - -- ensure the first time this blob sidecar was seen by a peer is in this table - -- 4294967295 = UInt32 max - `version` UInt32 DEFAULT 4294967295 - propagation_slot_start_diff COMMENT 'Version of this row, to help with de-duplication we want the latest updated_date_time but lowest propagation_slot_start_diff time' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The proposer index of the beacon block' CODEC(ZSTD(1)), - `blob_index` UInt32 COMMENT 'Blob index associated with the record' CODEC(ZSTD(1)), - `parent_root` FixedString(66) COMMENT 'Parent root of the beacon block' CODEC(ZSTD(1)), - `state_root` FixedString(66) COMMENT 'State root of the beacon block' CODEC(ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Network ID associated with the client' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - `version` -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) COMMENT 'Table for libp2p gossipsub blob sidecar data'; - -CREATE TABLE tmp.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' AS tmp.libp2p_gossipsub_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_gossipsub_blob_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) -); - -INSERT INTO - tmp.libp2p_gossipsub_blob_sidecar -SELECT - NOW(), - NULL, - event_date_time, - slot, - slot_start_date_time, - epoch, - epoch_start_date_time, - wallclock_slot, - wallclock_slot_start_date_time, - wallclock_epoch, - wallclock_epoch_start_date_time, - propagation_slot_start_diff, - proposer_index, - blob_index, - parent_root, - state_root, - peer_id_unique_key, - message_id, - message_size, - topic_layer, - topic_fork_digest_value, - topic_name, - topic_encoding, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_gossipsub_blob_sidecar_local; - -DROP TABLE IF EXISTS default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_gossipsub_blob_sidecar_local -AND tmp.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' AS default.libp2p_gossipsub_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_gossipsub_blob_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_handle_metadata -CREATE TABLE tmp.libp2p_handle_metadata_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the RPC', - `error` Nullable(String) COMMENT 'Error message if the metadata handling failed' CODEC(ZSTD(1)), - `protocol` LowCardinality(String) COMMENT 'The protocol of the metadata handling event', - `attnets` String COMMENT 'Attestation subnets the peer is subscribed to' CODEC(ZSTD(1)), - `seq_number` UInt64 COMMENT 'Sequence number of the metadata' CODEC(DoubleDelta, ZSTD(1)), - `syncnets` String COMMENT 'Sync subnets the peer is subscribed to' CODEC(ZSTD(1)), - `latency_milliseconds` Decimal(10, 3) COMMENT 'How long it took to handle the metadata request in milliseconds' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - attnets, - seq_number, - syncnets, - latency_milliseconds - ) COMMENT 'Contains the metadata handling events for libp2p peers.'; - -CREATE TABLE tmp.libp2p_handle_metadata ON CLUSTER '{cluster}' AS tmp.libp2p_handle_metadata_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_handle_metadata_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - attnets, - seq_number, - syncnets, - latency_milliseconds - ) -); - -INSERT INTO - tmp.libp2p_handle_metadata -SELECT - NOW(), - event_date_time, - peer_id_unique_key, - error, - protocol, - attnets, - seq_number, - syncnets, - latency_milliseconds, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_handle_metadata_local; - -DROP TABLE IF EXISTS default.libp2p_handle_metadata ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_handle_metadata_local -AND tmp.libp2p_handle_metadata_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_handle_metadata ON CLUSTER '{cluster}' AS default.libp2p_handle_metadata_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_handle_metadata_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - attnets, - seq_number, - syncnets, - latency_milliseconds - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_handle_metadata ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_handle_metadata_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_handle_status -CREATE TABLE tmp.libp2p_handle_status_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `error` Nullable(String) COMMENT 'Error message if the status handling failed' CODEC(ZSTD(1)), - `protocol` LowCardinality(String) COMMENT 'The protocol of the status handling event', - `request_finalized_epoch` Nullable(UInt32) COMMENT 'Requested finalized epoch' CODEC(DoubleDelta, ZSTD(1)), - `request_finalized_root` Nullable(String) COMMENT 'Requested finalized root', - `request_fork_digest` LowCardinality(String) COMMENT 'Requested fork digest', - `request_head_root` Nullable(FixedString(66)) COMMENT 'Requested head root' CODEC(ZSTD(1)), - `request_head_slot` Nullable(UInt32) COMMENT 'Requested head slot' CODEC(ZSTD(1)), - `response_finalized_epoch` Nullable(UInt32) COMMENT 'Response finalized epoch' CODEC(DoubleDelta, ZSTD(1)), - `response_finalized_root` Nullable(FixedString(66)) COMMENT 'Response finalized root' CODEC(ZSTD(1)), - `response_fork_digest` LowCardinality(String) COMMENT 'Response fork digest', - `response_head_root` Nullable(FixedString(66)) COMMENT 'Response head root' CODEC(ZSTD(1)), - `response_head_slot` Nullable(UInt32) COMMENT 'Response head slot' CODEC(DoubleDelta, ZSTD(1)), - `latency_milliseconds` Decimal(10, 3) COMMENT 'How long it took to handle the status request in milliseconds' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - latency_milliseconds - ) COMMENT 'Contains the status handling events for libp2p peers.'; - -CREATE TABLE tmp.libp2p_handle_status ON CLUSTER '{cluster}' AS tmp.libp2p_handle_status_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_handle_status_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - latency_milliseconds - ) -); - -INSERT INTO - tmp.libp2p_handle_status -SELECT - NOW(), - event_date_time, - peer_id_unique_key, - error, - protocol, - request_finalized_epoch, - request_finalized_root, - request_fork_digest, - request_head_root, - request_head_slot, - response_finalized_epoch, - response_finalized_root, - response_fork_digest, - response_head_root, - response_head_slot, - latency_milliseconds, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_handle_status_local; - -DROP TABLE IF EXISTS default.libp2p_handle_status ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_handle_status_local -AND tmp.libp2p_handle_status_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_handle_status ON CLUSTER '{cluster}' AS default.libp2p_handle_status_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_handle_status_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - latency_milliseconds - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_handle_status ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_handle_status_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_join -CREATE TABLE tmp.libp2p_join_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding of the topic', - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer that joined the topic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) COMMENT 'Contains the details of the JOIN events from the libp2p client.'; - -CREATE TABLE tmp.libp2p_join ON CLUSTER '{cluster}' AS tmp.libp2p_join_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_join_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) -); - -INSERT INTO - tmp.libp2p_join -SELECT - NOW(), - event_date_time, - topic_layer, - topic_fork_digest_value, - topic_name, - topic_encoding, - peer_id_unique_key, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_join_local; - -DROP TABLE IF EXISTS default.libp2p_join ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_join_local -AND tmp.libp2p_join_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_join ON CLUSTER '{cluster}' AS default.libp2p_join_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_join_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_join ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_join_local ON CLUSTER '{cluster}' SYNC; - --- libp2p_remove_peer -CREATE TABLE tmp.libp2p_remove_peer_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) COMMENT 'Contains the details of the peers removed from the libp2p client.'; - -CREATE TABLE tmp.libp2p_remove_peer ON CLUSTER '{cluster}' AS tmp.libp2p_remove_peer_local ENGINE = Distributed( - '{cluster}', - tmp, - libp2p_remove_peer_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) -); - -INSERT INTO - tmp.libp2p_remove_peer -SELECT - NOW(), - event_date_time, - peer_id_unique_key, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name -FROM - default.libp2p_remove_peer_local; - -DROP TABLE IF EXISTS default.libp2p_remove_peer ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.libp2p_remove_peer_local -AND tmp.libp2p_remove_peer_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.libp2p_remove_peer ON CLUSTER '{cluster}' AS default.libp2p_remove_peer_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_remove_peer_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key - ) -); - -DROP TABLE IF EXISTS tmp.libp2p_remove_peer ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.libp2p_remove_peer_local ON CLUSTER '{cluster}' SYNC; - --- mempool_dumpster_transaction -CREATE TABLE tmp.mempool_dumpster_transaction_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated, this is outside the source data and used for deduplication' CODEC(DoubleDelta, ZSTD(1)), - `timestamp` DateTime64(3) COMMENT 'Timestamp of the transaction' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), - `chain_id` UInt32 COMMENT 'The chain id of the transaction' CODEC(ZSTD(1)), - `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), - `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), - `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), - `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), - `gas_tip_cap` Nullable(UInt128) COMMENT 'The gas tip cap of the transaction in wei' CODEC(ZSTD(1)), - `gas_fee_cap` Nullable(UInt128) COMMENT 'The gas fee cap of the transaction in wei' CODEC(ZSTD(1)), - `data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `data_4bytes` Nullable(FixedString(10)) COMMENT 'The first 4 bytes of the call data of the transaction' CODEC(ZSTD(1)), - `sources` Array(LowCardinality(String)) COMMENT 'The sources that saw this transaction in their mempool', - `included_at_block_height` Nullable(UInt64) COMMENT 'The block height at which this transaction was included' CODEC(ZSTD(1)), - `included_block_timestamp` Nullable(DateTime64(3)) COMMENT 'The timestamp of the block at which this transaction was included' CODEC(DoubleDelta, ZSTD(1)), - `inclusion_delay_ms` Nullable(Int64) COMMENT 'The delay between the transaction timestamp and the block timestamp' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(timestamp) -ORDER BY - ( - timestamp, - chain_id, - hash, - `from`, - nonce, - gas - ) COMMENT 'Contains transactions from mempool dumpster dataset. Following the parquet schema with some additions'; - -CREATE TABLE tmp.mempool_dumpster_transaction ON CLUSTER '{cluster}' AS tmp.mempool_dumpster_transaction_local ENGINE = Distributed( - '{cluster}', - tmp, - mempool_dumpster_transaction_local, - cityHash64( - timestamp, - chain_id, - hash, - `from`, - nonce, - gas - ) -); - -INSERT INTO - tmp.mempool_dumpster_transaction -SELECT - NOW(), - timestamp, - hash, - chain_id, - `from`, - to, - value, - nonce, - gas, - gas_price, - gas_tip_cap, - gas_fee_cap, - data_size, - data_4bytes, - sources, - included_at_block_height, - included_block_timestamp, - inclusion_delay_ms -FROM - default.mempool_dumpster_transaction_local; - -DROP TABLE IF EXISTS default.mempool_dumpster_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.mempool_dumpster_transaction_local -AND tmp.mempool_dumpster_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.mempool_dumpster_transaction ON CLUSTER '{cluster}' AS default.mempool_dumpster_transaction_local ENGINE = Distributed( - '{cluster}', - default, - mempool_dumpster_transaction_local, - cityHash64( - timestamp, - chain_id, - hash, - `from`, - nonce, - gas - ) -); - -DROP TABLE IF EXISTS tmp.mempool_dumpster_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.mempool_dumpster_transaction_local ON CLUSTER '{cluster}' SYNC; - --- mempool_transaction -CREATE TABLE tmp.mempool_transaction_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'The time when the sentry saw the transaction in the mempool' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), - `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), - `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), - `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), - `gas_tip_cap` Nullable(UInt128) COMMENT 'The priority fee (tip) the user has set for the transaction', - `gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction', - `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), - `type` Nullable(UInt8) COMMENT 'The type of the transaction', - `size` UInt32 COMMENT 'The size of the transaction data in bytes' CODEC(ZSTD(1)), - `call_data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `blob_gas` Nullable(UInt64) COMMENT 'The maximum gas provided for the blob transaction execution', - `blob_gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction', - `blob_hashes` Array(String) COMMENT 'The hashes of the blob commitments for blob transactions', - `blob_sidecars_size` Nullable(UInt32) COMMENT 'The total size of the sidecars for blob transactions in bytes', - `blob_sidecars_empty_size` Nullable(UInt32) COMMENT 'The total empty size of the sidecars for blob transactions in bytes', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_execution_fork_id_hash` LowCardinality(String) COMMENT 'The hash of the fork ID of the current Ethereum network', - `meta_execution_fork_id_next` LowCardinality(String) COMMENT 'The fork ID of the next planned Ethereum network upgrade', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - hash, - `from`, - nonce, - gas - ) COMMENT 'Each row represents a transaction that was seen in the mempool by a sentry client. Sentries can report the same transaction multiple times if it has been long enough since the last report.'; - -CREATE TABLE tmp.mempool_transaction ON CLUSTER '{cluster}' AS tmp.mempool_transaction_local ENGINE = Distributed( - '{cluster}', - tmp, - mempool_transaction_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - hash, - `from`, - nonce, - gas - ) -); - -INSERT INTO - tmp.mempool_transaction -SELECT - NOW(), - event_date_time, - hash, - `from`, - to, - nonce, - gas_price, - gas, - gas_tip_cap, - gas_fee_cap, - value, - type, - size, - call_data_size, - blob_gas, - blob_gas_fee_cap, - blob_hashes, - blob_sidecars_size, - blob_sidecars_empty_size, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_execution_fork_id_hash, - meta_execution_fork_id_next, - meta_labels -FROM - default.mempool_transaction_local; - -DROP TABLE IF EXISTS default.mempool_transaction ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.mempool_transaction_local -AND tmp.mempool_transaction_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.mempool_transaction ON CLUSTER '{cluster}' AS default.mempool_transaction_local ENGINE = Distributed( - '{cluster}', - default, - mempool_transaction_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - hash, - `from`, - nonce, - gas - ) -); - -DROP TABLE IF EXISTS tmp.mempool_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.mempool_transaction_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/042_delete_beacon_p2p_attestation.down.sql b/deploy/migrations/clickhouse/042_delete_beacon_p2p_attestation.down.sql deleted file mode 100644 index 39dfb7ca..00000000 --- a/deploy/migrations/clickhouse/042_delete_beacon_p2p_attestation.down.sql +++ /dev/null @@ -1,95 +0,0 @@ -CREATE TABLE default.beacon_p2p_attestation_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node', - `slot` UInt32 COMMENT 'Slot number in the beacon P2P payload', - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started', - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon P2P payload', - `attesting_validator_index` Nullable(UInt32) COMMENT 'The index of the validator attesting to the event' CODEC(ZSTD(1)), - `attesting_validator_committee_index` LowCardinality(String) COMMENT 'The committee index of the attesting validator', - `aggregation_bits` String COMMENT 'The aggregation bits of the event in the beacon P2P payload' CODEC(ZSTD(1)), - `beacon_block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch` UInt32 COMMENT 'The source epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `source_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the source epoch started' CODEC(DoubleDelta, ZSTD(1)), - `source_root` FixedString(66) COMMENT 'The source beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `target_epoch` UInt32 COMMENT 'The target epoch number in the beacon P2P payload' CODEC(DoubleDelta, ZSTD(1)), - `target_epoch_start_date_time` DateTime COMMENT 'The wall clock time when the target epoch started' CODEC(DoubleDelta, ZSTD(1)), - `target_root` FixedString(66) COMMENT 'The target beacon block root hash in the beacon P2P payload' CODEC(ZSTD(1)), - `attestation_subnet` LowCardinality(String) COMMENT 'The attestation subnet the attestation was gossiped on', - `validated` Bool COMMENT 'Whether the attestation was validated by the client', - `peer_id` String COMMENT 'The originating peer ID for the gossiped data' CODEC(ZSTD(1)), - `peer_latency` UInt32 COMMENT 'The latency of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_version` LowCardinality(String) COMMENT 'Peer client version that gossiped the data', - `peer_version_major` LowCardinality(String) COMMENT 'Peer client major version that gossiped the data', - `peer_version_minor` LowCardinality(String) COMMENT 'Peer client minor version that gossiped the data', - `peer_version_patch` LowCardinality(String) COMMENT 'Peer client patch version that gossiped the data', - `peer_implementation` LowCardinality(String) COMMENT 'Peer client implementation that gossiped the data', - `peer_ip` Nullable(IPv6) COMMENT 'IP address of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_city` LowCardinality(String) COMMENT 'City of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_country` LowCardinality(String) COMMENT 'Country of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_country_code` LowCardinality(String) COMMENT 'Country code of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the peer that gossiped the data' CODEC(ZSTD(1)), - `peer_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the peer that gossiped the data' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - aggregation_bits, - beacon_block_root, - source_root, - target_root, - attestation_subnet, - peer_id - ) COMMENT 'Contains beacon chain P2P "attestation" data'; - -CREATE TABLE default.beacon_p2p_attestation ON CLUSTER '{cluster}' AS default.beacon_p2p_attestation_local ENGINE = Distributed( - '{cluster}', - default, - beacon_p2p_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - committee_index, - aggregation_bits, - beacon_block_root, - source_root, - target_root, - attestation_subnet, - peer_id - ) -); diff --git a/deploy/migrations/clickhouse/042_delete_beacon_p2p_attestation.up.sql b/deploy/migrations/clickhouse/042_delete_beacon_p2p_attestation.up.sql deleted file mode 100644 index 8ddc60a7..00000000 --- a/deploy/migrations/clickhouse/042_delete_beacon_p2p_attestation.up.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.beacon_p2p_attestation on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS default.beacon_p2p_attestation_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/043_canonical_beacon_validators_tweaks.down.sql b/deploy/migrations/clickhouse/043_canonical_beacon_validators_tweaks.down.sql deleted file mode 100644 index dbe0a8e4..00000000 --- a/deploy/migrations/clickhouse/043_canonical_beacon_validators_tweaks.down.sql +++ /dev/null @@ -1,56 +0,0 @@ -DROP TABLE IF EXISTS default.canonical_beacon_validators on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_beacon_validators_local on cluster '{cluster}' SYNC; - -CREATE TABLE default.canonical_beacon_validators_local -( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `index` UInt32 COMMENT 'The index of the validator' CODEC(ZSTD(1)), - `balance` UInt64 COMMENT 'The balance of the validator' CODEC(ZSTD(1)), - `status` LowCardinality(String) COMMENT 'The status of the validator', - `effective_balance` UInt64 COMMENT 'The effective balance of the validator' CODEC(ZSTD(1)), - `slashed` Bool COMMENT 'Whether the validator is slashed', - `activation_epoch` UInt64 COMMENT 'The epoch when the validator was activated' CODEC(ZSTD(1)), - `activation_eligibility_epoch` UInt64 COMMENT 'The epoch when the validator was activated' CODEC(ZSTD(1)), - `exit_epoch` UInt64 COMMENT 'The epoch when the validator exited' CODEC(ZSTD(1)), - `withdrawable_epoch` UInt64 COMMENT 'The epoch when the validator can withdraw' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) -ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY (epoch_start_date_time, `index`, meta_network_name) -COMMENT 'Contains a validator state for an epoch.' - -CREATE TABLE default.canonical_beacon_validators ON CLUSTER '{cluster}' AS default.canonical_beacon_validators_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_validators_local, - cityHash64( - epoch_start_date_time, - `index`, - meta_network_name - ) -); diff --git a/deploy/migrations/clickhouse/043_canonical_beacon_validators_tweaks.up.sql b/deploy/migrations/clickhouse/043_canonical_beacon_validators_tweaks.up.sql deleted file mode 100644 index fd47e771..00000000 --- a/deploy/migrations/clickhouse/043_canonical_beacon_validators_tweaks.up.sql +++ /dev/null @@ -1,68 +0,0 @@ -DROP TABLE IF EXISTS default.canonical_beacon_validators on cluster '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_validators_local on cluster '{cluster}' SYNC; - -TRUNCATE TABLE canonical_beacon_validators_pubkeys_local ON CLUSTER '{cluster}'; - -TRUNCATE TABLE canonical_beacon_validators_withdrawal_credentials_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.canonical_beacon_validators_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'When this row was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number from beacon block payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `index` UInt32 COMMENT 'The index of the validator' CODEC(DoubleDelta, ZSTD(1)), - `balance` Nullable(UInt64) COMMENT 'The balance of the validator' CODEC(T64, ZSTD(1)), - `status` LowCardinality(String) COMMENT 'The status of the validator', - `effective_balance` Nullable(UInt64) COMMENT 'The effective balance of the validator' CODEC(ZSTD(1)), - `slashed` Bool COMMENT 'Whether the validator is slashed', - `activation_epoch` Nullable(UInt64) COMMENT 'The epoch when the validator was activated' CODEC(ZSTD(1)), - `activation_eligibility_epoch` Nullable(UInt64) COMMENT 'The epoch when the validator was activated' CODEC(ZSTD(1)), - `exit_epoch` Nullable(UInt64) COMMENT 'The epoch when the validator exited' CODEC(ZSTD(1)), - `withdrawable_epoch` Nullable(UInt64) COMMENT 'The epoch when the validator can withdraw' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY - ( - epoch_start_date_time, - meta_network_name, - `index`, - `status` - ) COMMENT 'Contains a validator state for an epoch.'; - -CREATE TABLE default.canonical_beacon_validators ON CLUSTER '{cluster}' AS default.canonical_beacon_validators_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_validators_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - `index`, - `status` - ) -); diff --git a/deploy/migrations/clickhouse/044_events_voluntary_exit_wallclock.down.sql b/deploy/migrations/clickhouse/044_events_voluntary_exit_wallclock.down.sql deleted file mode 100644 index 16cf0238..00000000 --- a/deploy/migrations/clickhouse/044_events_voluntary_exit_wallclock.down.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `validator_index` UInt32 COMMENT 'The index of the validator making the voluntary exit', - `signature` String COMMENT 'The signature of the voluntary exit in the beacon API event stream payload', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY - ( - epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_voluntary_exit_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_voluntary_exit -SELECT - NOW(), - event_date_time, - epoch, - epoch_start_date_time, - validator_index, - signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_voluntary_exit_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_voluntary_exit_local -AND tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_voluntary_exit_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/044_events_voluntary_exit_wallclock.up.sql b/deploy/migrations/clickhouse/044_events_voluntary_exit_wallclock.up.sql deleted file mode 100644 index 061127cf..00000000 --- a/deploy/migrations/clickhouse/044_events_voluntary_exit_wallclock.up.sql +++ /dev/null @@ -1,124 +0,0 @@ -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload', - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started', - `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `validator_index` UInt32 COMMENT 'The index of the validator making the voluntary exit', - `signature` String COMMENT 'The signature of the voluntary exit in the beacon API event stream payload', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event', - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event', - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event', - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event', - `meta_network_id` Int32 COMMENT 'Ethereum network ID', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tmp/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(wallclock_epoch_start_date_time) -ORDER BY - ( - wallclock_epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.'; - -CREATE TABLE tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS tmp.beacon_api_eth_v1_events_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - tmp, - beacon_api_eth_v1_events_voluntary_exit_local, - cityHash64( - wallclock_epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) -); - -INSERT INTO - tmp.beacon_api_eth_v1_events_voluntary_exit -WITH - (CASE - WHEN meta_network_name = 'mainnet' THEN 1606824023 - WHEN meta_network_name = 'sepolia' THEN 1655733600 - WHEN meta_network_name = 'holesky' THEN 1695902400 - WHEN meta_network_name = 'dencun-msf-1' THEN 1708607100 - ELSE 0 - END) AS genesis_time -SELECT - NOW(), - event_date_time, - epoch, - toDateTime(genesis_time + (epoch * 12 * 32)) AS epoch_start_date_time, - floor((toUnixTimestamp(event_date_time) - genesis_time) / 12) AS wallclock_slot, - toDateTime(genesis_time + (floor((toUnixTimestamp(event_date_time) - genesis_time) / 12) * 12)) AS wallclock_slot_start_date_time, - floor((toUnixTimestamp(event_date_time) - genesis_time) / 12 / 32) AS wallclock_epoch, - toDateTime(genesis_time + (floor((toUnixTimestamp(event_date_time) - genesis_time) / 12 / 32) * 12 * 32)) AS wallclock_epoch_start_date_time, - validator_index, - signature, - meta_client_name, - meta_client_id, - meta_client_version, - meta_client_implementation, - meta_client_os, - meta_client_ip, - meta_client_geo_city, - meta_client_geo_country, - meta_client_geo_country_code, - meta_client_geo_continent_code, - meta_client_geo_longitude, - meta_client_geo_latitude, - meta_client_geo_autonomous_system_number, - meta_client_geo_autonomous_system_organization, - meta_network_id, - meta_network_name, - meta_consensus_version, - meta_consensus_version_major, - meta_consensus_version_minor, - meta_consensus_version_patch, - meta_consensus_implementation, - meta_labels -FROM - default.beacon_api_eth_v1_events_voluntary_exit_local; - -DROP TABLE IF EXISTS default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -EXCHANGE TABLES default.beacon_api_eth_v1_events_voluntary_exit_local -AND tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}'; - -CREATE TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_voluntary_exit_local, - cityHash64( - wallclock_epoch_start_date_time, - meta_network_name, - meta_client_name, - validator_index - ) -); - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS tmp.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/045_canonical_beacon_committee.down.sql b/deploy/migrations/clickhouse/045_canonical_beacon_committee.down.sql deleted file mode 100644 index 9a682443..00000000 --- a/deploy/migrations/clickhouse/045_canonical_beacon_committee.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.canonical_beacon_committee ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_beacon_committee_local ON cluster '{cluster}' SYNC; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/045_canonical_beacon_committee.up.sql b/deploy/migrations/clickhouse/045_canonical_beacon_committee.up.sql deleted file mode 100644 index f41c5c45..00000000 --- a/deploy/migrations/clickhouse/045_canonical_beacon_committee.up.sql +++ /dev/null @@ -1,53 +0,0 @@ -CREATE TABLE default.canonical_beacon_committee_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `committee_index` LowCardinality(String) COMMENT 'The committee index in the beacon API committee payload', - `validators` Array(UInt32) COMMENT 'The validator indices in the beacon API committee payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API committee payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - committee_index, - ) COMMENT 'Contains canonical beacon API /eth/v1/beacon/committees data.'; - -CREATE TABLE default.canonical_beacon_committee ON CLUSTER '{cluster}' AS default.canonical_beacon_committee_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_committee_local, - cityHash64( - slot_start_date_time, - meta_network_name, - committee_index - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/046_remove_canonical_event_time.down.sql b/deploy/migrations/clickhouse/046_remove_canonical_event_time.down.sql deleted file mode 100644 index 77244a81..00000000 --- a/deploy/migrations/clickhouse/046_remove_canonical_event_time.down.sql +++ /dev/null @@ -1,294 +0,0 @@ -DROP TABLE IF EXISTS default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_validators ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_validators_pubkeys ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_validators_withdrawal_credentials ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_committee ON CLUSTER '{cluster}' SYNC; - -ALTER TABLE - default.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_block_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_validators_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_validators_pubkeys_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_validators_withdrawal_credentials_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE - default.canonical_beacon_committee_local ON CLUSTER '{cluster}' -ADD - COLUMN event_date_time DateTime64(3) COMMENT 'When the client fetched the data from the beacon node' CODEC(DoubleDelta, ZSTD(1)); - -CREATE TABLE default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' AS default.canonical_beacon_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_blob_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - blob_index - ) -); - -CREATE TABLE default.canonical_beacon_block ON CLUSTER '{cluster}' AS default.canonical_beacon_block_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_local, - cityHash64( - slot_start_date_time, - meta_network_name - ) -); - -CREATE TABLE default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' AS default.canonical_beacon_block_attester_slashing_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_attester_slashing_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - attestation_1_attesting_indices, - attestation_2_attesting_indices, - attestation_1_data_slot, - attestation_2_data_slot, - attestation_1_data_beacon_block_root, - attestation_2_data_beacon_block_root - ) -); - -CREATE TABLE default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' AS default.canonical_beacon_block_bls_to_execution_change_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_bls_to_execution_change_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - exchanging_message_validator_index, - exchanging_message_from_bls_pubkey, - exchanging_message_to_execution_address - ) -); - -CREATE TABLE default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' AS default.canonical_beacon_block_deposit_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_deposit_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - deposit_data_pubkey, - deposit_proof - ) -); - -CREATE TABLE default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' AS default.canonical_beacon_block_execution_transaction_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_execution_transaction_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - position, - hash, - nonce - ) -); - -CREATE TABLE default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' AS default.canonical_beacon_block_proposer_slashing_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_proposer_slashing_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - signed_header_1_message_slot, - signed_header_2_message_slot, - signed_header_1_message_proposer_index, - signed_header_2_message_proposer_index, - signed_header_1_message_body_root, - signed_header_2_message_body_root - ) -); - -CREATE TABLE default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' AS default.canonical_beacon_block_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_voluntary_exit_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - voluntary_exit_message_epoch, - voluntary_exit_message_validator_index - ) -); - -CREATE TABLE default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' AS default.canonical_beacon_block_withdrawal_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_withdrawal_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - withdrawal_index, - withdrawal_validator_index - ) -); - -CREATE TABLE default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' AS default.canonical_beacon_elaborated_attestation_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_elaborated_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - block_slot, - position_in_block, - beacon_block_root, - slot, - committee_index, - source_root, - target_root - ) -); - -CREATE TABLE default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' AS default.canonical_beacon_proposer_duty_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_proposer_duty_local, - cityHash64( - slot_start_date_time, - meta_network_name, - proposer_validator_index, - proposer_pubkey - ) -); - -CREATE TABLE default.canonical_beacon_validators ON CLUSTER '{cluster}' AS default.canonical_beacon_validators_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_validators_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - `index`, - `status` - ) -); - -CREATE TABLE default.canonical_beacon_validators_pubkeys on cluster '{cluster}' AS default.canonical_beacon_validators_pubkeys_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_validators_pubkeys_local, - cityHash64(`index`, meta_network_name) -); - -CREATE TABLE default.canonical_beacon_validators_withdrawal_credentials on cluster '{cluster}' AS default.canonical_beacon_validators_withdrawal_credentials_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_validators_withdrawal_credentials_local, - cityHash64(`index`, meta_network_name) -); - -CREATE TABLE default.canonical_beacon_committee ON CLUSTER '{cluster}' AS default.canonical_beacon_committee_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_committee_local, - cityHash64( - slot_start_date_time, - meta_network_name, - committee_index - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/046_remove_canonical_event_time.up.sql b/deploy/migrations/clickhouse/046_remove_canonical_event_time.up.sql deleted file mode 100644 index 0fadf779..00000000 --- a/deploy/migrations/clickhouse/046_remove_canonical_event_time.up.sql +++ /dev/null @@ -1,264 +0,0 @@ -DROP TABLE IF EXISTS default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_validators ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_validators_pubkeys ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_validators_withdrawal_credentials ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_beacon_committee ON CLUSTER '{cluster}' SYNC; - -ALTER TABLE - default.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_block_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_block_bls_to_execution_change_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_block_proposer_slashing_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_validators_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_validators_pubkeys_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_validators_withdrawal_credentials_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -ALTER TABLE - default.canonical_beacon_committee_local ON CLUSTER '{cluster}' DROP COLUMN event_date_time; - -CREATE TABLE default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' AS default.canonical_beacon_blob_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_blob_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - blob_index - ) -) COMMENT 'Contains a blob sidecar from a beacon block.'; - -CREATE TABLE default.canonical_beacon_block ON CLUSTER '{cluster}' AS default.canonical_beacon_block_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_local, - cityHash64( - slot_start_date_time, - meta_network_name - ) -) COMMENT 'Contains beacon block from a beacon node.'; - -CREATE TABLE default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' AS default.canonical_beacon_block_attester_slashing_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_attester_slashing_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - attestation_1_attesting_indices, - attestation_2_attesting_indices, - attestation_1_data_slot, - attestation_2_data_slot, - attestation_1_data_beacon_block_root, - attestation_2_data_beacon_block_root - ) -) COMMENT 'Contains attester slashing from a beacon block.'; - -CREATE TABLE default.canonical_beacon_block_bls_to_execution_change ON CLUSTER '{cluster}' AS default.canonical_beacon_block_bls_to_execution_change_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_bls_to_execution_change_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - exchanging_message_validator_index, - exchanging_message_from_bls_pubkey, - exchanging_message_to_execution_address - ) -) COMMENT 'Contains bls to execution change from a beacon block.'; - -CREATE TABLE default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' AS default.canonical_beacon_block_deposit_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_deposit_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - deposit_data_pubkey, - deposit_proof - ) -) COMMENT 'Contains a deposit from a beacon block.'; - -CREATE TABLE default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' AS default.canonical_beacon_block_execution_transaction_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_execution_transaction_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - position, - hash, - nonce - ) -) COMMENT 'Contains execution transaction from a beacon block.'; - -CREATE TABLE default.canonical_beacon_block_proposer_slashing ON CLUSTER '{cluster}' AS default.canonical_beacon_block_proposer_slashing_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_proposer_slashing_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - signed_header_1_message_slot, - signed_header_2_message_slot, - signed_header_1_message_proposer_index, - signed_header_2_message_proposer_index, - signed_header_1_message_body_root, - signed_header_2_message_body_root - ) -) COMMENT 'Contains proposer slashing from a beacon block.'; - -CREATE TABLE default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' AS default.canonical_beacon_block_voluntary_exit_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_voluntary_exit_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - voluntary_exit_message_epoch, - voluntary_exit_message_validator_index - ) -) COMMENT 'Contains a voluntary exit from a beacon block.'; - -CREATE TABLE default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' AS default.canonical_beacon_block_withdrawal_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_withdrawal_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - withdrawal_index, - withdrawal_validator_index - ) -) COMMENT 'Contains a withdrawal from a beacon block.'; - -CREATE TABLE default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' AS default.canonical_beacon_elaborated_attestation_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_elaborated_attestation_local, - cityHash64( - slot_start_date_time, - meta_network_name, - block_root, - block_slot, - position_in_block, - beacon_block_root, - slot, - committee_index, - source_root, - target_root - ) -) COMMENT 'Contains elaborated attestations from beacon blocks.'; - -CREATE TABLE default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' AS default.canonical_beacon_proposer_duty_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_proposer_duty_local, - cityHash64( - slot_start_date_time, - meta_network_name, - proposer_validator_index, - proposer_pubkey - ) -) COMMENT 'Contains a proposer duty from a beacon block.'; - -CREATE TABLE default.canonical_beacon_validators ON CLUSTER '{cluster}' AS default.canonical_beacon_validators_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_validators_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - `index`, - `status` - ) -) COMMENT 'Contains a validator state for an epoch.'; - -CREATE TABLE default.canonical_beacon_validators_pubkeys on cluster '{cluster}' AS default.canonical_beacon_validators_pubkeys_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_validators_pubkeys_local, - cityHash64(`index`, meta_network_name) -) COMMENT 'Contains a validator pubkeys for an epoch.'; - -CREATE TABLE default.canonical_beacon_validators_withdrawal_credentials on cluster '{cluster}' AS default.canonical_beacon_validators_withdrawal_credentials_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_validators_withdrawal_credentials_local, - cityHash64(`index`, meta_network_name) -) COMMENT 'Contains a validator withdrawal credentials for an epoch.'; - -CREATE TABLE default.canonical_beacon_committee ON CLUSTER '{cluster}' AS default.canonical_beacon_committee_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_committee_local, - cityHash64( - slot_start_date_time, - meta_network_name, - committee_index - ) -) COMMENT 'Contains canonical beacon API /eth/v1/beacon/committees data.'; diff --git a/deploy/migrations/clickhouse/047_block_execution_payloads.down.sql b/deploy/migrations/clickhouse/047_block_execution_payloads.down.sql deleted file mode 100644 index 9aaff2b4..00000000 --- a/deploy/migrations/clickhouse/047_block_execution_payloads.down.sql +++ /dev/null @@ -1,27 +0,0 @@ -ALTER TABLE canonical_beacon_block ON CLUSTER '{cluster}' - DROP COLUMN execution_payload_base_fee_per_gas, - DROP COLUMN execution_payload_blob_gas_used, - DROP COLUMN execution_payload_excess_blob_gas, - DROP COLUMN execution_payload_gas_limit, - DROP COLUMN execution_payload_gas_used; - -ALTER TABLE canonical_beacon_block_local ON CLUSTER '{cluster}' - DROP COLUMN execution_payload_base_fee_per_gas, - DROP COLUMN execution_payload_blob_gas_used, - DROP COLUMN execution_payload_excess_blob_gas, - DROP COLUMN execution_payload_gas_limit, - DROP COLUMN execution_payload_gas_used; - -ALTER TABLE beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' - DROP COLUMN execution_payload_base_fee_per_gas, - DROP COLUMN execution_payload_blob_gas_used, - DROP COLUMN execution_payload_excess_blob_gas, - DROP COLUMN execution_payload_gas_limit, - DROP COLUMN execution_payload_gas_used; - -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' - DROP COLUMN execution_payload_base_fee_per_gas, - DROP COLUMN execution_payload_blob_gas_used, - DROP COLUMN execution_payload_excess_blob_gas, - DROP COLUMN execution_payload_gas_limit, - DROP COLUMN execution_payload_gas_used; diff --git a/deploy/migrations/clickhouse/047_block_execution_payloads.up.sql b/deploy/migrations/clickhouse/047_block_execution_payloads.up.sql deleted file mode 100644 index e1f6afd5..00000000 --- a/deploy/migrations/clickhouse/047_block_execution_payloads.up.sql +++ /dev/null @@ -1,27 +0,0 @@ -ALTER TABLE canonical_beacon_block_local ON CLUSTER '{cluster}' - ADD COLUMN execution_payload_base_fee_per_gas Nullable(UInt128) COMMENT 'Base fee per gas for execution payload' CODEC(ZSTD(1)) AFTER execution_payload_fee_recipient, - ADD COLUMN execution_payload_blob_gas_used Nullable(UInt64) COMMENT 'Gas used for blobs in execution payload' CODEC(ZSTD(1)) AFTER execution_payload_base_fee_per_gas, - ADD COLUMN execution_payload_excess_blob_gas Nullable(UInt64) COMMENT 'Excess gas used for blobs in execution payload' CODEC(ZSTD(1)) AFTER execution_payload_blob_gas_used, - ADD COLUMN execution_payload_gas_limit Nullable(UInt64) COMMENT 'Gas limit for execution payload' CODEC(DoubleDelta, ZSTD(1)) AFTER execution_payload_excess_blob_gas, - ADD COLUMN execution_payload_gas_used Nullable(UInt64) COMMENT 'Gas used for execution payload' CODEC(ZSTD(1)) AFTER execution_payload_gas_limit; - -ALTER TABLE canonical_beacon_block ON CLUSTER '{cluster}' - ADD COLUMN execution_payload_base_fee_per_gas Nullable(UInt128) COMMENT 'Base fee per gas for execution payload' CODEC(ZSTD(1)) AFTER execution_payload_fee_recipient, - ADD COLUMN execution_payload_blob_gas_used Nullable(UInt64) COMMENT 'Gas used for blobs in execution payload' CODEC(ZSTD(1)) AFTER execution_payload_base_fee_per_gas, - ADD COLUMN execution_payload_excess_blob_gas Nullable(UInt64) COMMENT 'Excess gas used for blobs in execution payload' CODEC(ZSTD(1)) AFTER execution_payload_blob_gas_used, - ADD COLUMN execution_payload_gas_limit Nullable(UInt64) COMMENT 'Gas limit for execution payload' CODEC(DoubleDelta, ZSTD(1)) AFTER execution_payload_excess_blob_gas, - ADD COLUMN execution_payload_gas_used Nullable(UInt64) COMMENT 'Gas used for execution payload' CODEC(ZSTD(1)) AFTER execution_payload_gas_limit; - -ALTER TABLE beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' - ADD COLUMN execution_payload_base_fee_per_gas Nullable(UInt128) COMMENT 'Base fee per gas for execution payload' CODEC(ZSTD(1)) AFTER execution_payload_fee_recipient, - ADD COLUMN execution_payload_blob_gas_used Nullable(UInt64) COMMENT 'Gas used for blobs in execution payload' CODEC(ZSTD(1)) AFTER execution_payload_base_fee_per_gas, - ADD COLUMN execution_payload_excess_blob_gas Nullable(UInt64) COMMENT 'Excess gas used for blobs in execution payload' CODEC(ZSTD(1)) AFTER execution_payload_blob_gas_used, - ADD COLUMN execution_payload_gas_limit Nullable(UInt64) COMMENT 'Gas limit for execution payload' CODEC(DoubleDelta, ZSTD(1)) AFTER execution_payload_excess_blob_gas, - ADD COLUMN execution_payload_gas_used Nullable(UInt64) COMMENT 'Gas used for execution payload' CODEC(ZSTD(1)) AFTER execution_payload_gas_limit; - -ALTER TABLE beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' - ADD COLUMN execution_payload_base_fee_per_gas Nullable(UInt128) COMMENT 'Base fee per gas for execution payload' CODEC(ZSTD(1)) AFTER execution_payload_fee_recipient, - ADD COLUMN execution_payload_blob_gas_used Nullable(UInt64) COMMENT 'Gas used for blobs in execution payload' CODEC(ZSTD(1)) AFTER execution_payload_base_fee_per_gas, - ADD COLUMN execution_payload_excess_blob_gas Nullable(UInt64) COMMENT 'Excess gas used for blobs in execution payload' CODEC(ZSTD(1)) AFTER execution_payload_blob_gas_used, - ADD COLUMN execution_payload_gas_limit Nullable(UInt64) COMMENT 'Gas limit for execution payload' CODEC(DoubleDelta, ZSTD(1)) AFTER execution_payload_excess_blob_gas, - ADD COLUMN execution_payload_gas_used Nullable(UInt64) COMMENT 'Gas used for execution payload' CODEC(ZSTD(1)) AFTER execution_payload_gas_limit; diff --git a/deploy/migrations/clickhouse/048_nullable_canonical_beacon_block_execution.down.sql b/deploy/migrations/clickhouse/048_nullable_canonical_beacon_block_execution.down.sql deleted file mode 100644 index 92da6f21..00000000 --- a/deploy/migrations/clickhouse/048_nullable_canonical_beacon_block_execution.down.sql +++ /dev/null @@ -1,13 +0,0 @@ -ALTER TABLE default.canonical_beacon_block ON CLUSTER '{cluster}' - MODIFY COLUMN `execution_payload_block_hash` FixedString(66) COMMENT 'The block hash of the execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_block_number` UInt32 COMMENT 'The block number of the execution payload' CODEC(DoubleDelta, ZSTD(1)), - MODIFY COLUMN `execution_payload_fee_recipient` String COMMENT 'The recipient of the fee for this execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_state_root` FixedString(66) COMMENT 'The state root of the execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_parent_hash` FixedString(66) COMMENT 'The parent hash of the execution payload' CODEC(ZSTD(1)); - -ALTER TABLE default.canonical_beacon_block_local ON CLUSTER '{cluster}' - MODIFY COLUMN `execution_payload_block_hash` FixedString(66) COMMENT 'The block hash of the execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_block_number` UInt32 COMMENT 'The block number of the execution payload' CODEC(DoubleDelta, ZSTD(1)), - MODIFY COLUMN `execution_payload_fee_recipient` String COMMENT 'The recipient of the fee for this execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_state_root` FixedString(66) COMMENT 'The state root of the execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_parent_hash` FixedString(66) COMMENT 'The parent hash of the execution payload' CODEC(ZSTD(1)); diff --git a/deploy/migrations/clickhouse/048_nullable_canonical_beacon_block_execution.up.sql b/deploy/migrations/clickhouse/048_nullable_canonical_beacon_block_execution.up.sql deleted file mode 100644 index 37455d25..00000000 --- a/deploy/migrations/clickhouse/048_nullable_canonical_beacon_block_execution.up.sql +++ /dev/null @@ -1,13 +0,0 @@ -ALTER TABLE default.canonical_beacon_block_local ON CLUSTER '{cluster}' - MODIFY COLUMN `execution_payload_block_hash` Nullable(FixedString(66)) COMMENT 'The block hash of the execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_block_number` Nullable(UInt32) COMMENT 'The block number of the execution payload' CODEC(DoubleDelta, ZSTD(1)), - MODIFY COLUMN `execution_payload_fee_recipient` Nullable(String) COMMENT 'The recipient of the fee for this execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_state_root` Nullable(FixedString(66)) COMMENT 'The state root of the execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_parent_hash` Nullable(FixedString(66)) COMMENT 'The parent hash of the execution payload' CODEC(ZSTD(1)); - -ALTER TABLE default.canonical_beacon_block ON CLUSTER '{cluster}' - MODIFY COLUMN `execution_payload_block_hash` Nullable(FixedString(66)) COMMENT 'The block hash of the execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_block_number` Nullable(UInt32) COMMENT 'The block number of the execution payload' CODEC(DoubleDelta, ZSTD(1)), - MODIFY COLUMN `execution_payload_fee_recipient` Nullable(String) COMMENT 'The recipient of the fee for this execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_state_root` Nullable(FixedString(66)) COMMENT 'The state root of the execution payload' CODEC(ZSTD(1)), - MODIFY COLUMN `execution_payload_parent_hash` Nullable(FixedString(66)) COMMENT 'The parent hash of the execution payload' CODEC(ZSTD(1)); diff --git a/deploy/migrations/clickhouse/049_canonical_execution.down.sql b/deploy/migrations/clickhouse/049_canonical_execution.down.sql deleted file mode 100644 index 3e0a9e7c..00000000 --- a/deploy/migrations/clickhouse/049_canonical_execution.down.sql +++ /dev/null @@ -1,34 +0,0 @@ -DROP TABLE IF EXISTS default.canonical_execution_block ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_block_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_transaction ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_transaction_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_balance_diffs ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_balance_diffs_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_balance_reads ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_balance_reads_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_four_byte_counts ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_four_byte_counts_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_erc20_transfers ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_erc20_transfers_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_erc721_transfers ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_erc721_transfers_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_native_transfers ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_native_transfers_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_nonce_diffs ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_nonce_diffs_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_nonce_reads ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_nonce_reads_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_address_appearances ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_address_appearances_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_contracts ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_contracts_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_traces ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_traces_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_logs ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_logs_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_storage_diffs ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_storage_diffs_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_storage_reads ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_storage_reads_local ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS admin.cryo ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS admin.cryo_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/049_canonical_execution.up.sql b/deploy/migrations/clickhouse/049_canonical_execution.up.sql deleted file mode 100644 index 2e963b58..00000000 --- a/deploy/migrations/clickhouse/049_canonical_execution.up.sql +++ /dev/null @@ -1,634 +0,0 @@ -CREATE DATABASE IF NOT EXISTS `admin` ON CLUSTER '{cluster}'; - -CREATE TABLE admin.cryo_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), - `dataset` LowCardinality(String), - `mode` LowCardinality(String), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) -ORDER BY - ( - dataset, - mode, - meta_network_name - ); - -CREATE TABLE admin.cryo ON CLUSTER '{cluster}' AS admin.cryo_local ENGINE = Distributed( - '{cluster}', - admin, - cryo_local, - cityHash64( - dataset, - mode, - meta_network_name - ) -); - -CREATE TABLE default.canonical_execution_block_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_date_time` DateTime64(3) COMMENT 'The block timestamp' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `block_hash` FixedString(66) COMMENT 'The block hash' CODEC(ZSTD(1)), - `author` Nullable(String) COMMENT 'The block author' CODEC(ZSTD(1)), - `gas_used` Nullable(UInt64) COMMENT 'The block gas used' CODEC(DoubleDelta, ZSTD(1)), - `extra_data` Nullable(String) COMMENT 'The block extra data in hex' CODEC(ZSTD(1)), - `extra_data_string` Nullable(String) COMMENT 'The block extra data in UTF-8 string' CODEC(ZSTD(1)), - `base_fee_per_gas` Nullable(UInt64) COMMENT 'The block base fee per gas' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - ) COMMENT 'Contains canonical execution block data.'; - -CREATE TABLE default.canonical_execution_block ON CLUSTER '{cluster}' AS default.canonical_execution_block_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_block_local, - cityHash64( - block_number, - meta_network_name - ) -); - -CREATE TABLE default.canonical_execution_transaction_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The transaction nonce' CODEC(ZSTD(1)), - `from_address` String COMMENT 'The transaction from address' CODEC(ZSTD(1)), - `to_address` Nullable(String) COMMENT 'The transaction to address' CODEC(ZSTD(1)), - `value` UInt256 COMMENT 'The transaction value in float64' CODEC(ZSTD(1)), - `input` Nullable(String) COMMENT 'The transaction input in hex' CODEC(ZSTD(1)), - `gas_limit` UInt64 COMMENT 'The transaction gas limit' CODEC(ZSTD(1)), - `gas_used` UInt64 COMMENT 'The transaction gas used' CODEC(ZSTD(1)), - `gas_price` UInt64 COMMENT 'The transaction gas price' CODEC(ZSTD(1)), - `transaction_type` UInt32 COMMENT 'The transaction type' CODEC(ZSTD(1)), - `max_priority_fee_per_gas` UInt64 COMMENT 'The transaction max priority fee per gas' CODEC(ZSTD(1)), - `max_fee_per_gas` UInt64 COMMENT 'The transaction max fee per gas' CODEC(ZSTD(1)), - `success` Bool COMMENT 'The transaction success' CODEC(ZSTD(1)), - `n_input_bytes` UInt32 COMMENT 'The transaction input bytes' CODEC(ZSTD(1)), - `n_input_zero_bytes` UInt32 COMMENT 'The transaction input zero bytes' CODEC(ZSTD(1)), - `n_input_nonzero_bytes` UInt32 COMMENT 'The transaction input nonzero bytes' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash - ) COMMENT 'Contains canonical execution transaction data.'; - -CREATE TABLE default.canonical_execution_transaction ON CLUSTER '{cluster}' AS default.canonical_execution_transaction_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_transaction_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash - ) -); - -CREATE TABLE default.canonical_execution_balance_diffs_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the balance diff' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the balance diff within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `address` String COMMENT 'The address of the balance diff' CODEC(ZSTD(1)), - `from_value` UInt256 COMMENT 'The from value of the balance diff' CODEC(ZSTD(1)), - `to_value` UInt256 COMMENT 'The to value of the balance diff' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution balance diff data.'; - -CREATE TABLE default.canonical_execution_balance_diffs ON CLUSTER '{cluster}' AS default.canonical_execution_balance_diffs_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_balance_diffs_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_balance_reads_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the balance read' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the balance read within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `address` String COMMENT 'The address of the balance read' CODEC(ZSTD(1)), - `balance` UInt256 COMMENT 'The balance that was read' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution balance read data.'; - -CREATE TABLE default.canonical_execution_balance_reads ON CLUSTER '{cluster}' AS default.canonical_execution_balance_reads_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_balance_reads_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_four_byte_counts_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), - `signature` String COMMENT 'The signature of the four byte count' CODEC(ZSTD(1)), - `size` UInt64 COMMENT 'The size of the four byte count' CODEC(ZSTD(1)), - `count` UInt64 COMMENT 'The count of the four byte count' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash - ) COMMENT 'Contains canonical execution four byte count data.'; - -CREATE TABLE default.canonical_execution_four_byte_counts ON CLUSTER '{cluster}' AS default.canonical_execution_four_byte_counts_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_four_byte_counts_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash - ) -); - -CREATE TABLE default.canonical_execution_erc20_transfers_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the transfer within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `log_index` UInt64 COMMENT 'The log index in the block' CODEC(DoubleDelta, ZSTD(1)), - `erc20` String COMMENT 'The erc20 address' CODEC(ZSTD(1)), - `from_address` String COMMENT 'The from address' CODEC(ZSTD(1)), - `to_address` String COMMENT 'The to address' CODEC(ZSTD(1)), - `value` UInt256 COMMENT 'The value of the transfer' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution erc20 transfer data.'; - -CREATE TABLE default.canonical_execution_erc20_transfers ON CLUSTER '{cluster}' AS default.canonical_execution_erc20_transfers_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_erc20_transfers_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_erc721_transfers_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the transfer within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `log_index` UInt64 COMMENT 'The log index in the block' CODEC(DoubleDelta, ZSTD(1)), - `erc20` String COMMENT 'The erc20 address' CODEC(ZSTD(1)), - `from_address` String COMMENT 'The from address' CODEC(ZSTD(1)), - `to_address` String COMMENT 'The to address' CODEC(ZSTD(1)), - `token` UInt256 COMMENT 'The token id' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution erc721 transfer data.'; - -CREATE TABLE default.canonical_execution_erc721_transfers ON CLUSTER '{cluster}' AS default.canonical_execution_erc721_transfers_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_erc721_transfers_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_native_transfers_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the transfer within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `transfer_index` UInt64 COMMENT 'The transfer index' CODEC(DoubleDelta, ZSTD(1)), - `from_address` String COMMENT 'The from address' CODEC(ZSTD(1)), - `to_address` String COMMENT 'The to address' CODEC(ZSTD(1)), - `value` UInt256 COMMENT 'The value of the approval' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution native transfer data.'; - -CREATE TABLE default.canonical_execution_native_transfers ON CLUSTER '{cluster}' AS default.canonical_execution_native_transfers_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_native_transfers_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_nonce_diffs_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the nonce diff' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the nonce diff within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `address` String COMMENT 'The address of the nonce diff' CODEC(ZSTD(1)), - `from_value` UInt64 COMMENT 'The from value of the nonce diff' CODEC(ZSTD(1)), - `to_value` UInt64 COMMENT 'The to value of the nonce diff' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution nonce diff data.'; - -CREATE TABLE default.canonical_execution_nonce_diffs ON CLUSTER '{cluster}' AS default.canonical_execution_nonce_diffs_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_nonce_diffs_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_nonce_reads_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt64 COMMENT 'The transaction index in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the nonce read' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the nonce read within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `address` String COMMENT 'The address of the nonce read' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The nonce that was read' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution nonce read data.'; - -CREATE TABLE default.canonical_execution_nonce_reads ON CLUSTER '{cluster}' AS default.canonical_execution_nonce_reads_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_nonce_reads_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_address_appearances_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash that caused the address appearance' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the address appearance within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `address` String COMMENT 'The address of the address appearance' CODEC(ZSTD(1)), - `relationship` LowCardinality(String) COMMENT 'The relationship of the address to the transaction', - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution address appearance data.'; - -CREATE TABLE default.canonical_execution_address_appearances ON CLUSTER '{cluster}' AS default.canonical_execution_address_appearances_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_address_appearances_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_contracts_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt32 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash that created the contract' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the contract creation within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `create_index` UInt32 COMMENT 'The create index' CODEC(DoubleDelta, ZSTD(1)), - `contract_address` String COMMENT 'The contract address' CODEC(ZSTD(1)), - `deployer` String COMMENT 'The address of the contract deployer' CODEC(ZSTD(1)), - `factory` String COMMENT 'The address of the factory that deployed the contract' CODEC(ZSTD(1)), - `init_code` String COMMENT 'The initialization code of the contract' CODEC(ZSTD(1)), - `code` Nullable(String) COMMENT 'The code of the contract' CODEC(ZSTD(1)), - `init_code_hash` String COMMENT 'The hash of the initialization code' CODEC(ZSTD(1)), - `n_init_code_bytes` UInt32 COMMENT 'Number of bytes in the initialization code' CODEC(DoubleDelta, ZSTD(1)), - `n_code_bytes` UInt32 COMMENT 'Number of bytes in the contract code' CODEC(DoubleDelta, ZSTD(1)), - `code_hash` String COMMENT 'The hash of the contract code' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution contract data.'; - -CREATE TABLE default.canonical_execution_contracts ON CLUSTER '{cluster}' AS default.canonical_execution_contracts_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_contracts_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_traces_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt32 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt32 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the trace within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `action_from` String COMMENT 'The from address of the action' CODEC(ZSTD(1)), - `action_to` Nullable(String) COMMENT 'The to address of the action' CODEC(ZSTD(1)), - `action_value` String COMMENT 'The value of the action' CODEC(ZSTD(1)), - `action_gas` UInt32 COMMENT 'The gas provided for the action' CODEC(DoubleDelta, ZSTD(1)), - `action_input` Nullable(String) COMMENT 'The input data for the action' CODEC(ZSTD(1)), - `action_call_type` LowCardinality(String) COMMENT 'The call type of the action' CODEC(ZSTD(1)), - `action_init` Nullable(String) COMMENT 'The initialization code for the action' CODEC(ZSTD(1)), - `action_reward_type` String COMMENT 'The reward type for the action' CODEC(ZSTD(1)), - `action_type` LowCardinality(String) COMMENT 'The type of the action' CODEC(ZSTD(1)), - `result_gas_used` UInt32 COMMENT 'The gas used in the result' CODEC(DoubleDelta, ZSTD(1)), - `result_output` Nullable(String) COMMENT 'The output of the result' CODEC(ZSTD(1)), - `result_code` Nullable(String) COMMENT 'The code returned in the result' CODEC(ZSTD(1)), - `result_address` Nullable(String) COMMENT 'The address returned in the result' CODEC(ZSTD(1)), - `trace_address` Nullable(String) COMMENT 'The trace address' CODEC(ZSTD(1)), - `subtraces` UInt32 COMMENT 'The number of subtraces' CODEC(DoubleDelta, ZSTD(1)), - `error` Nullable(String) COMMENT 'The error, if any, in the trace' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution traces data.'; - -CREATE TABLE default.canonical_execution_traces ON CLUSTER '{cluster}' AS default.canonical_execution_traces_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_traces_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_logs_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt32 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt32 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash associated with the log' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the log within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `log_index` UInt32 COMMENT 'The log index within the block' CODEC(DoubleDelta, ZSTD(1)), - `address` String COMMENT 'The address associated with the log' CODEC(ZSTD(1)), - `topic0` String COMMENT 'The first topic of the log' CODEC(ZSTD(1)), - `topic1` Nullable(String) COMMENT 'The second topic of the log' CODEC(ZSTD(1)), - `topic2` Nullable(String) COMMENT 'The third topic of the log' CODEC(ZSTD(1)), - `topic3` Nullable(String) COMMENT 'The fourth topic of the log' CODEC(ZSTD(1)), - `data` Nullable(String) COMMENT 'The data associated with the log' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution logs data.'; - -CREATE TABLE default.canonical_execution_logs ON CLUSTER '{cluster}' AS default.canonical_execution_logs_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_logs_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_storage_diffs_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt32 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt32 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash associated with the storage diff' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the storage diff within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `address` String COMMENT 'The address associated with the storage diff' CODEC(ZSTD(1)), - `slot` String COMMENT 'The storage slot key' CODEC(ZSTD(1)), - `from_value` String COMMENT 'The original value before the storage diff' CODEC(ZSTD(1)), - `to_value` String COMMENT 'The new value after the storage diff' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution storage diffs data.'; - -CREATE TABLE default.canonical_execution_storage_diffs ON CLUSTER '{cluster}' AS default.canonical_execution_storage_diffs_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_storage_diffs_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); - -CREATE TABLE default.canonical_execution_storage_reads_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt32 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_index` UInt32 COMMENT 'The transaction index' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash associated with the storage read' CODEC(ZSTD(1)), - `internal_index` UInt32 COMMENT 'The internal index of the storage read within the transaction' CODEC(DoubleDelta, ZSTD(1)), - `contract_address` String COMMENT 'The contract address associated with the storage read' CODEC(ZSTD(1)), - `slot` String COMMENT 'The storage slot key' CODEC(ZSTD(1)), - `value` String COMMENT 'The value read from the storage slot' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) COMMENT 'Contains canonical execution storage reads data.'; - -CREATE TABLE default.canonical_execution_storage_reads ON CLUSTER '{cluster}' AS default.canonical_execution_storage_reads_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_storage_reads_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - internal_index - ) -); diff --git a/deploy/migrations/clickhouse/050_ethseer_validator_entity.down.sql b/deploy/migrations/clickhouse/050_ethseer_validator_entity.down.sql deleted file mode 100644 index ad2ffc9b..00000000 --- a/deploy/migrations/clickhouse/050_ethseer_validator_entity.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.ethseer_validator_entity ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.ethseer_validator_entity_local ON CLUSTER '{cluster}'; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/050_ethseer_validator_entity.up.sql b/deploy/migrations/clickhouse/050_ethseer_validator_entity.up.sql deleted file mode 100644 index 9da73082..00000000 --- a/deploy/migrations/clickhouse/050_ethseer_validator_entity.up.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE default.ethseer_validator_entity_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the client fetched the beacon block from ethseer.io' CODEC(DoubleDelta, ZSTD(1)), - `index` UInt32 COMMENT 'The index of the validator' CODEC(DoubleDelta, ZSTD(1)), - `pubkey` String COMMENT 'The public key of the validator' CODEC(ZSTD(1)), - `entity` String COMMENT 'The entity of the validator' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) -ORDER BY - (index, pubkey, meta_network_name) COMMENT 'Contains a mapping of validators to entities'; - -CREATE TABLE default.ethseer_validator_entity ON CLUSTER '{cluster}' AS default.ethseer_validator_entity_local ENGINE = Distributed( - '{cluster}', - default, - ethseer_validator_entity_local, - cityHash64(`index`, pubkey, meta_network_name) -); diff --git a/deploy/migrations/clickhouse/051_mev_relay_block_bids.down.sql b/deploy/migrations/clickhouse/051_mev_relay_block_bids.down.sql deleted file mode 100644 index 02e1cb16..00000000 --- a/deploy/migrations/clickhouse/051_mev_relay_block_bids.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS mev_relay_bid_trace on cluster '{cluster}' SYNC; -DROP TABLE IF EXISTS mev_relay_bid_trace_local on cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/051_mev_relay_block_bids.up.sql b/deploy/migrations/clickhouse/051_mev_relay_block_bids.up.sql deleted file mode 100644 index f88dfc91..00000000 --- a/deploy/migrations/clickhouse/051_mev_relay_block_bids.up.sql +++ /dev/null @@ -1,68 +0,0 @@ -CREATE TABLE default.mev_relay_bid_trace_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the bid was fetched' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number within the block bid' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The start time for the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number derived from the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The start time for the epoch that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_request_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_request_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_request_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_request_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `requested_at_slot_time` UInt32 COMMENT 'The time in the slot when the request was sent' CODEC(ZSTD(1)), - `response_at_slot_time` UInt32 COMMENT 'The time in the slot when the response was received' CODEC(ZSTD(1)), - `relay_name` String COMMENT 'The relay that the bid was fetched from' CODEC(ZSTD(1)), - `parent_hash` FixedString(66) COMMENT 'The parent hash of the bid' CODEC(ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number of the bid' CODEC(DoubleDelta, ZSTD(1)), - `block_hash` FixedString(66) COMMENT 'The block hash of the bid' CODEC(ZSTD(1)), - `builder_pubkey` String COMMENT 'The builder pubkey of the bid' CODEC(ZSTD(1)), - `proposer_pubkey` String COMMENT 'The proposer pubkey of the bid' CODEC(ZSTD(1)), - `proposer_fee_recipient` FixedString(42) COMMENT 'The proposer fee recipient of the bid' CODEC(ZSTD(1)), - `gas_limit` UInt64 COMMENT 'The gas limit of the bid' CODEC(DoubleDelta, ZSTD(1)), - `gas_used` UInt64 COMMENT 'The gas used of the bid' CODEC(DoubleDelta, ZSTD(1)), - `value` UInt256 COMMENT 'The transaction value in float64' CODEC(ZSTD(1)), - `num_tx` UInt32 COMMENT 'The number of transactions in the bid' CODEC(DoubleDelta, ZSTD(1)), - `timestamp` Int64 COMMENT 'The timestamp of the bid' CODEC(DoubleDelta, ZSTD(1)), - `timestamp_ms` Int64 COMMENT 'The timestamp of the bid in milliseconds' CODEC(DoubleDelta, ZSTD(1)), - `optimistic_submission` Bool COMMENT 'Whether the bid was optimistic' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - slot, - block_hash, - meta_client_name, - builder_pubkey, - proposer_pubkey - ) COMMENT 'Contains MEV relay block bids data.'; - -CREATE TABLE default.mev_relay_bid_trace ON CLUSTER '{cluster}' AS default.mev_relay_bid_trace_local ENGINE = Distributed( - '{cluster}', - default, - mev_relay_bid_trace_local, - cityHash64( - slot, - meta_network_name - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/052_mev_relay_proposer_payload_delivered.down.sql b/deploy/migrations/clickhouse/052_mev_relay_proposer_payload_delivered.down.sql deleted file mode 100644 index 2119eb9c..00000000 --- a/deploy/migrations/clickhouse/052_mev_relay_proposer_payload_delivered.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS mev_relay_proposer_payload_delivered ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS mev_relay_proposer_payload_delivered_local ON CLUSTER '{cluster}' SYNC; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/052_mev_relay_proposer_payload_delivered.up.sql b/deploy/migrations/clickhouse/052_mev_relay_proposer_payload_delivered.up.sql deleted file mode 100644 index 6d235fc5..00000000 --- a/deploy/migrations/clickhouse/052_mev_relay_proposer_payload_delivered.up.sql +++ /dev/null @@ -1,61 +0,0 @@ -CREATE TABLE default.mev_relay_proposer_payload_delivered_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the payload was delivered' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number within the payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The start time for the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number derived from the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The start time for the epoch that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number of the payload' CODEC(DoubleDelta, ZSTD(1)), - `relay_name` String COMMENT 'The relay that delivered the payload' CODEC(ZSTD(1)), - `block_hash` FixedString(66) COMMENT 'The block hash associated with the payload' CODEC(ZSTD(1)), - `proposer_pubkey` String COMMENT 'The proposer pubkey that received the payload' CODEC(ZSTD(1)), - `builder_pubkey` String COMMENT 'The builder pubkey that sent the payload' CODEC(ZSTD(1)), - `proposer_fee_recipient` FixedString(42) COMMENT 'The proposer fee recipient of the payload' CODEC(ZSTD(1)), - `gas_limit` UInt64 COMMENT 'The gas limit of the payload' CODEC(DoubleDelta, ZSTD(1)), - `gas_used` UInt64 COMMENT 'The gas used by the payload' CODEC(DoubleDelta, ZSTD(1)), - `num_tx` UInt32 COMMENT 'The number of transactions in the payload' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - relay_name, - block_hash, - meta_client_name, - builder_pubkey, - proposer_pubkey, - ) COMMENT 'Contains MEV relay proposer payload delivered data.'; - -CREATE TABLE default.mev_relay_proposer_payload_delivered ON CLUSTER '{cluster}' AS default.mev_relay_proposer_payload_delivered_local ENGINE = Distributed( - '{cluster}', - default, - mev_relay_proposer_payload_delivered_local, - cityHash64( - slot, - meta_network_name - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/053_add_bid_trace_relay_name_unique.down.sql b/deploy/migrations/clickhouse/053_add_bid_trace_relay_name_unique.down.sql deleted file mode 100644 index faa7c6bc..00000000 --- a/deploy/migrations/clickhouse/053_add_bid_trace_relay_name_unique.down.sql +++ /dev/null @@ -1,5 +0,0 @@ --- Drop the distributed table -DROP TABLE IF EXISTS default.mev_relay_bid_trace ON CLUSTER '{cluster}' SYNC; - --- Drop the local table -DROP TABLE IF EXISTS default.mev_relay_bid_trace_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/053_add_bid_trace_relay_name_unique.up.sql b/deploy/migrations/clickhouse/053_add_bid_trace_relay_name_unique.up.sql deleted file mode 100644 index 0271dcc0..00000000 --- a/deploy/migrations/clickhouse/053_add_bid_trace_relay_name_unique.up.sql +++ /dev/null @@ -1,72 +0,0 @@ --- Step 1: Drop the existing tables -DROP TABLE IF EXISTS default.mev_relay_bid_trace ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.mev_relay_bid_trace_local ON CLUSTER '{cluster}' SYNC; - --- Step 2: Create the new table with updated ORDER BY key -CREATE TABLE default.mev_relay_bid_trace_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the bid was fetched' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number within the block bid' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The start time for the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number derived from the slot that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The start time for the epoch that the bid is for' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_request_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_request_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_request_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_request_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `requested_at_slot_time` UInt32 COMMENT 'The time in the slot when the request was sent' CODEC(ZSTD(1)), - `response_at_slot_time` UInt32 COMMENT 'The time in the slot when the response was received' CODEC(ZSTD(1)), - `relay_name` String COMMENT 'The relay that the bid was fetched from' CODEC(ZSTD(1)), - `parent_hash` FixedString(66) COMMENT 'The parent hash of the bid' CODEC(ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number of the bid' CODEC(DoubleDelta, ZSTD(1)), - `block_hash` FixedString(66) COMMENT 'The block hash of the bid' CODEC(ZSTD(1)), - `builder_pubkey` String COMMENT 'The builder pubkey of the bid' CODEC(ZSTD(1)), - `proposer_pubkey` String COMMENT 'The proposer pubkey of the bid' CODEC(ZSTD(1)), - `proposer_fee_recipient` FixedString(42) COMMENT 'The proposer fee recipient of the bid' CODEC(ZSTD(1)), - `gas_limit` UInt64 COMMENT 'The gas limit of the bid' CODEC(DoubleDelta, ZSTD(1)), - `gas_used` UInt64 COMMENT 'The gas used of the bid' CODEC(DoubleDelta, ZSTD(1)), - `value` UInt256 COMMENT 'The transaction value in float64' CODEC(ZSTD(1)), - `num_tx` UInt32 COMMENT 'The number of transactions in the bid' CODEC(DoubleDelta, ZSTD(1)), - `timestamp` Int64 COMMENT 'The timestamp of the bid' CODEC(DoubleDelta, ZSTD(1)), - `timestamp_ms` Int64 COMMENT 'The timestamp of the bid in milliseconds' CODEC(DoubleDelta, ZSTD(1)), - `optimistic_submission` Bool COMMENT 'Whether the bid was optimistic' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - relay_name, -- Add relay_name to the ORDER BY key since the same block can be offered through multiple relays - block_hash, - meta_client_name, - builder_pubkey, - proposer_pubkey - ) COMMENT 'Contains MEV relay block bids data.'; - --- Step 3: Create the new distributed table -CREATE TABLE default.mev_relay_bid_trace ON CLUSTER '{cluster}' AS default.mev_relay_bid_trace_local ENGINE = Distributed( - '{cluster}', - default, - mev_relay_bid_trace_local, - cityHash64(slot, meta_network_name) -); diff --git a/deploy/migrations/clickhouse/054_validator_block.down.sql b/deploy/migrations/clickhouse/054_validator_block.down.sql deleted file mode 100644 index 7971d5fb..00000000 --- a/deploy/migrations/clickhouse/054_validator_block.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v3_validator_block ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v3_validator_block_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/054_validator_block.up.sql b/deploy/migrations/clickhouse/054_validator_block.up.sql deleted file mode 100644 index 3028a9ab..00000000 --- a/deploy/migrations/clickhouse/054_validator_block.up.sql +++ /dev/null @@ -1,70 +0,0 @@ --- Step 1: Create the new table. -CREATE TABLE default.beacon_api_eth_v3_validator_block_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number within the payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the reorg slot started', - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `block_total_bytes` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload' CODEC(ZSTD(1)), - `block_total_bytes_compressed` Nullable(UInt32) COMMENT 'The total bytes of the beacon block payload when compressed using snappy' CODEC(ZSTD(1)), - `consensus_payload_value` Nullable(UInt64) COMMENT 'Consensus rewards paid to the proposer for this block, in Wei. Use to determine relative value of consensus blocks.' CODEC(ZSTD(1)), - `execution_payload_value` Nullable(UInt64) COMMENT 'Execution payload value in Wei. Use to determine relative value of execution payload.' CODEC(ZSTD(1)), - `execution_payload_block_number` UInt32 COMMENT 'The block number of the execution payload', - `execution_payload_base_fee_per_gas` Nullable(UInt128) COMMENT 'Base fee per gas for execution payload' CODEC(ZSTD(1)), - `execution_payload_blob_gas_used` Nullable(UInt64) COMMENT 'Gas used for blobs in execution payload' CODEC(ZSTD(1)), - `execution_payload_excess_blob_gas` Nullable(UInt64) COMMENT 'Excess gas used for blobs in execution payload' CODEC(ZSTD(1)), - `execution_payload_gas_limit` Nullable(UInt64) COMMENT 'Gas limit for execution payload' CODEC(DoubleDelta, ZSTD(1)), - `execution_payload_gas_used` Nullable(UInt64) COMMENT 'Gas used for execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_count` Nullable(UInt32) COMMENT 'The transaction count of the execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_total_bytes` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload' CODEC(ZSTD(1)), - `execution_payload_transactions_total_bytes_compressed` Nullable(UInt32) COMMENT 'The transaction total bytes of the execution payload when compressed using snappy' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event', - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event', - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event', - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event', - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event', - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY -( - slot_start_date_time, - meta_network_name, - meta_client_name, - event_date_time -) -COMMENT 'Contains beacon API /eth/v3/validator/blocks/{slot} data from each sentry client attached to a beacon node.'; - --- Step 2: Create the distributed table. -CREATE TABLE default.beacon_api_eth_v3_validator_block ON CLUSTER '{cluster}' AS default.beacon_api_eth_v3_validator_block_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v3_validator_block_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - event_date_time - ) -); diff --git a/deploy/migrations/clickhouse/055_mev_relay_validator_registrations.down.sql b/deploy/migrations/clickhouse/055_mev_relay_validator_registrations.down.sql deleted file mode 100644 index 6df73bec..00000000 --- a/deploy/migrations/clickhouse/055_mev_relay_validator_registrations.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS mev_relay_validator_registration ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS mev_relay_validator_registration_local ON CLUSTER '{cluster}' SYNC; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/055_mev_relay_validator_registrations.up.sql b/deploy/migrations/clickhouse/055_mev_relay_validator_registrations.up.sql deleted file mode 100644 index f69411b6..00000000 --- a/deploy/migrations/clickhouse/055_mev_relay_validator_registrations.up.sql +++ /dev/null @@ -1,56 +0,0 @@ -CREATE TABLE default.mev_relay_validator_registration_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the bid was fetched' CODEC(DoubleDelta, ZSTD(1)), - `timestamp` Int64 COMMENT 'The timestamp of the bid' CODEC(DoubleDelta, ZSTD(1)), - `relay_name` String COMMENT 'The relay that the bid was fetched from' CODEC(ZSTD(1)), - `validator_index` UInt32 COMMENT 'The validator index of the validator registration' CODEC(ZSTD(1)), - `gas_limit` UInt64 COMMENT 'The gas limit of the validator registration' CODEC(DoubleDelta, ZSTD(1)), - `fee_recipient` String COMMENT 'The fee recipient of the validator registration' CODEC(ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The slot start time derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The epoch start time derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - relay_name, - validator_index, - timestamp - ) COMMENT 'Contains MEV relay validator registrations data.'; - -CREATE TABLE default.mev_relay_validator_registration ON CLUSTER '{cluster}' AS default.mev_relay_validator_registration_local ENGINE = Distributed( - '{cluster}', - default, - mev_relay_validator_registration_local, - cityHash64( - slot, - meta_network_name - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/056_event_timestamp_ordered_validator_registrations.down.sql b/deploy/migrations/clickhouse/056_event_timestamp_ordered_validator_registrations.down.sql deleted file mode 100644 index fcd29e40..00000000 --- a/deploy/migrations/clickhouse/056_event_timestamp_ordered_validator_registrations.down.sql +++ /dev/null @@ -1,59 +0,0 @@ -DROP TABLE IF EXISTS mev_relay_validator_registration ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS mev_relay_validator_registration_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE default.mev_relay_validator_registration_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the bid was fetched' CODEC(DoubleDelta, ZSTD(1)), - `timestamp` Int64 COMMENT 'The timestamp of the bid' CODEC(DoubleDelta, ZSTD(1)), - `relay_name` String COMMENT 'The relay that the bid was fetched from' CODEC(ZSTD(1)), - `validator_index` UInt32 COMMENT 'The validator index of the validator registration' CODEC(ZSTD(1)), - `gas_limit` UInt64 COMMENT 'The gas limit of the validator registration' CODEC(DoubleDelta, ZSTD(1)), - `fee_recipient` String COMMENT 'The fee recipient of the validator registration' CODEC(ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The slot start time derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The epoch start time derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - relay_name, - validator_index, - timestamp - ) COMMENT 'Contains MEV relay validator registrations data.'; - -CREATE TABLE default.mev_relay_validator_registration ON CLUSTER '{cluster}' AS default.mev_relay_validator_registration_local ENGINE = Distributed( - '{cluster}', - default, - mev_relay_validator_registration_local, - cityHash64( - slot, - meta_network_name - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/056_event_timestamp_ordered_validator_registrations.up.sql b/deploy/migrations/clickhouse/056_event_timestamp_ordered_validator_registrations.up.sql deleted file mode 100644 index 04b14e24..00000000 --- a/deploy/migrations/clickhouse/056_event_timestamp_ordered_validator_registrations.up.sql +++ /dev/null @@ -1,59 +0,0 @@ -DROP TABLE IF EXISTS mev_relay_validator_registration ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS mev_relay_validator_registration_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE IF NOT EXISTS mev_relay_validator_registration_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the registration was fetched' CODEC(DoubleDelta, ZSTD(1)), - `timestamp` Int64 COMMENT 'The timestamp of the registration' CODEC(DoubleDelta, ZSTD(1)), - `relay_name` String COMMENT 'The relay that the registration was fetched from' CODEC(ZSTD(1)), - `validator_index` UInt32 COMMENT 'The validator index of the validator registration' CODEC(ZSTD(1)), - `gas_limit` UInt64 COMMENT 'The gas limit of the validator registration' CODEC(DoubleDelta, ZSTD(1)), - `fee_recipient` String COMMENT 'The fee recipient of the validator registration' CODEC(ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The slot start time derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The epoch start time derived from the validator registration `timestamp` field' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - relay_name, - validator_index, - timestamp - ) COMMENT 'Contains MEV relay validator registrations data.'; - -CREATE TABLE mev_relay_validator_registration ON CLUSTER '{cluster}' AS mev_relay_validator_registration_local ENGINE = Distributed( - '{cluster}', - default, - mev_relay_validator_registration_local, - cityHash64( - slot, - meta_network_name - ) -); diff --git a/deploy/migrations/clickhouse/057_block_gossip.down.sql b/deploy/migrations/clickhouse/057_block_gossip.down.sql deleted file mode 100644 index 70b79607..00000000 --- a/deploy/migrations/clickhouse/057_block_gossip.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_events_block_gossip ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_events_block_gossip_local ON CLUSTER '{cluster}' SYNC; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/057_block_gossip.up.sql b/deploy/migrations/clickhouse/057_block_gossip.up.sql deleted file mode 100644 index 93427cfc..00000000 --- a/deploy/migrations/clickhouse/057_block_gossip.up.sql +++ /dev/null @@ -1,55 +0,0 @@ -CREATE TABLE default.beacon_api_eth_v1_events_block_gossip_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `block` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - block - ) COMMENT 'Contains beacon API eventstream "block_gossip" data from each sentry client attached to a beacon node.'; - -CREATE TABLE default.beacon_api_eth_v1_events_block_gossip ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_block_gossip_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_block_gossip_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block - ) -); diff --git a/deploy/migrations/clickhouse/058_libp2p_drop_rpc.down.sql b/deploy/migrations/clickhouse/058_libp2p_drop_rpc.down.sql deleted file mode 100644 index 67820d90..00000000 --- a/deploy/migrations/clickhouse/058_libp2p_drop_rpc.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS libp2p_drop_rpc ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS libp2p_drop_rpc_local ON CLUSTER '{cluster}' SYNC; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/058_libp2p_drop_rpc.up.sql b/deploy/migrations/clickhouse/058_libp2p_drop_rpc.up.sql deleted file mode 100644 index d7e3e7e4..00000000 --- a/deploy/migrations/clickhouse/058_libp2p_drop_rpc.up.sql +++ /dev/null @@ -1,43 +0,0 @@ --- libp2p_drop_rpc_local --- unique_key = seahash(.event.id) --- peer_id_unique_key = seahash(data.meta.peer_id + .meta_network_name) --- --- Control messages (prune, iwant, idontwant, etc) are split off from this message --- and stored in separate tables: --- libp2p_rpc_meta_control_prune_local --- libp2p_rpc_meta_control_iwant_local --- libp2p_rpc_meta_control_idontwant_local --- etc. --- --- These control tables reference: --- rpc_meta_unique_key = libp2p_*_rpc_local.unique_key --- peer_id_unique_key = libp2p_*_rpc_local.peer_id_unique_key -CREATE TABLE libp2p_drop_rpc_local ON CLUSTER '{cluster}' -( - unique_key Int64 COMMENT 'Unique identifier for each record', - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - peer_id_unique_key Int64 COMMENT 'Unique key associated with the identifier of the peer receiver', - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, meta_network_name, meta_client_name) -COMMENT 'Contains the details of the RPC messages dropped by the peer.'; - -CREATE TABLE libp2p_drop_rpc ON CLUSTER '{cluster}' AS libp2p_drop_rpc_local -ENGINE = Distributed('{cluster}', default, libp2p_drop_rpc_local, unique_key); diff --git a/deploy/migrations/clickhouse/059_libp2p_trace_leave.down.sql b/deploy/migrations/clickhouse/059_libp2p_trace_leave.down.sql deleted file mode 100644 index 41901422..00000000 --- a/deploy/migrations/clickhouse/059_libp2p_trace_leave.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_leave ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.libp2p_leave_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/059_libp2p_trace_leave.up.sql b/deploy/migrations/clickhouse/059_libp2p_trace_leave.up.sql deleted file mode 100644 index 05f209ff..00000000 --- a/deploy/migrations/clickhouse/059_libp2p_trace_leave.up.sql +++ /dev/null @@ -1,53 +0,0 @@ -CREATE TABLE libp2p_leave_local ON CLUSTER '{cluster}' -( - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String) COMMENT 'Layer of the topic', - topic_fork_digest_value LowCardinality(String) COMMENT 'Fork digest value of the topic', - topic_name LowCardinality(String) COMMENT 'Name of the topic', - topic_encoding LowCardinality(String) COMMENT 'Encoding of the topic', - peer_id_unique_key Int64 COMMENT 'Unique key associated with the identifier of the peer that left the topic', - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) COMMENT 'Contains the details of the LEAVE events from the libp2p client.'; - -CREATE TABLE libp2p_leave ON CLUSTER '{cluster}' AS libp2p_leave_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_leave_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/060_libp2p_trace_graft.down.sql b/deploy/migrations/clickhouse/060_libp2p_trace_graft.down.sql deleted file mode 100644 index 90ab7349..00000000 --- a/deploy/migrations/clickhouse/060_libp2p_trace_graft.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_graft ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.libp2p_graft_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/060_libp2p_trace_graft.up.sql b/deploy/migrations/clickhouse/060_libp2p_trace_graft.up.sql deleted file mode 100644 index f4387cf2..00000000 --- a/deploy/migrations/clickhouse/060_libp2p_trace_graft.up.sql +++ /dev/null @@ -1,53 +0,0 @@ -CREATE TABLE libp2p_graft_local ON CLUSTER '{cluster}' -( - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String) COMMENT 'Layer of the topic', - topic_fork_digest_value LowCardinality(String) COMMENT 'Fork digest value of the topic', - topic_name LowCardinality(String) COMMENT 'Name of the topic', - topic_encoding LowCardinality(String) COMMENT 'Encoding of the topic', - peer_id_unique_key Int64 COMMENT 'Unique key for the peer that initiated the GRAFT (eg joined the mesh for this topic) identifies mesh membership changes per peer.', - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) COMMENT 'Contains the details of the GRAFT events from the libp2p client.'; - -CREATE TABLE libp2p_graft ON CLUSTER '{cluster}' AS libp2p_graft_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_graft_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/061_libp2p_trace_prune.down.sql b/deploy/migrations/clickhouse/061_libp2p_trace_prune.down.sql deleted file mode 100644 index 9175ba7e..00000000 --- a/deploy/migrations/clickhouse/061_libp2p_trace_prune.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_prune ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.libp2p_prune_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/061_libp2p_trace_prune.up.sql b/deploy/migrations/clickhouse/061_libp2p_trace_prune.up.sql deleted file mode 100644 index 94dd8573..00000000 --- a/deploy/migrations/clickhouse/061_libp2p_trace_prune.up.sql +++ /dev/null @@ -1,53 +0,0 @@ -CREATE TABLE libp2p_prune_local ON CLUSTER '{cluster}' -( - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String) COMMENT 'Layer of the topic', - topic_fork_digest_value LowCardinality(String) COMMENT 'Fork digest value of the topic', - topic_name LowCardinality(String) COMMENT 'Name of the topic', - topic_encoding LowCardinality(String) COMMENT 'Encoding of the topic', - peer_id_unique_key Int64 COMMENT 'Unique key for the peer that was PRUNED (eg removed from the mesh for this topic) identifies mesh membership changes per peer.', - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) COMMENT 'Contains the details of the PRUNE events from the libp2p client.'; - -CREATE TABLE libp2p_prune ON CLUSTER '{cluster}' AS libp2p_prune_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_prune_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/062_libp2p_trace_deliver_message.down.sql b/deploy/migrations/clickhouse/062_libp2p_trace_deliver_message.down.sql deleted file mode 100644 index ee1a7cb7..00000000 --- a/deploy/migrations/clickhouse/062_libp2p_trace_deliver_message.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_deliver_message ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.libp2p_deliver_message_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/062_libp2p_trace_deliver_message.up.sql b/deploy/migrations/clickhouse/062_libp2p_trace_deliver_message.up.sql deleted file mode 100644 index ed99691d..00000000 --- a/deploy/migrations/clickhouse/062_libp2p_trace_deliver_message.up.sql +++ /dev/null @@ -1,61 +0,0 @@ -CREATE TABLE libp2p_deliver_message_local ON CLUSTER '{cluster}' -( - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String) COMMENT 'Layer of the topic', - topic_fork_digest_value LowCardinality(String) COMMENT 'Fork digest value of the topic', - topic_name LowCardinality(String) COMMENT 'Name of the topic', - topic_encoding LowCardinality(String) COMMENT 'Encoding of the topic', - seq_number UInt64 COMMENT 'A linearly increasing number that is unique among messages originating from the given peer' CODEC(DoubleDelta, ZSTD(1)), - local_delivery Bool COMMENT 'Indicates if the message was delivered to in-process subscribers only', - peer_id_unique_key Int64 COMMENT 'Unique key for the peer that delivered the message', - message_id String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - message_size UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name, - message_id, - seq_number - ) COMMENT 'Contains the details of the DELIVER_MESSAGE events from the libp2p client.'; - -CREATE TABLE libp2p_deliver_message ON CLUSTER '{cluster}' AS libp2p_deliver_message_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_deliver_message_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name, - message_id, - seq_number - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/063_libp2p_trace_reject_message.down.sql b/deploy/migrations/clickhouse/063_libp2p_trace_reject_message.down.sql deleted file mode 100644 index 2fdbb608..00000000 --- a/deploy/migrations/clickhouse/063_libp2p_trace_reject_message.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_reject_message ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.libp2p_reject_message_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/063_libp2p_trace_reject_message.up.sql b/deploy/migrations/clickhouse/063_libp2p_trace_reject_message.up.sql deleted file mode 100644 index d2885bab..00000000 --- a/deploy/migrations/clickhouse/063_libp2p_trace_reject_message.up.sql +++ /dev/null @@ -1,62 +0,0 @@ -CREATE TABLE libp2p_reject_message_local ON CLUSTER '{cluster}' -( - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String) COMMENT 'Layer of the topic', - topic_fork_digest_value LowCardinality(String) COMMENT 'Fork digest value of the topic', - topic_name LowCardinality(String) COMMENT 'Name of the topic', - topic_encoding LowCardinality(String) COMMENT 'Encoding of the topic', - seq_number UInt64 COMMENT 'A linearly increasing number that is unique among messages originating from the given peer' CODEC(DoubleDelta, ZSTD(1)), - local_delivery Bool COMMENT 'Indicates if the message was rejected by local subscriber', - peer_id_unique_key Int64 COMMENT 'Unique key for the peer that rejected the message', - message_id String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - message_size UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - reason String COMMENT 'Reason for message rejection' CODEC(ZSTD(1)), - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name, - message_id, - seq_number - ) COMMENT 'Contains the details of the REJECT_MESSAGE events from the libp2p client.'; - -CREATE TABLE libp2p_reject_message ON CLUSTER '{cluster}' AS libp2p_reject_message_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_reject_message_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name, - message_id, - seq_number - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/064_libp2p_trace_publish_message.down.sql b/deploy/migrations/clickhouse/064_libp2p_trace_publish_message.down.sql deleted file mode 100644 index 17843101..00000000 --- a/deploy/migrations/clickhouse/064_libp2p_trace_publish_message.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_publish_message ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.libp2p_publish_message_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/064_libp2p_trace_publish_message.up.sql b/deploy/migrations/clickhouse/064_libp2p_trace_publish_message.up.sql deleted file mode 100644 index d054ae65..00000000 --- a/deploy/migrations/clickhouse/064_libp2p_trace_publish_message.up.sql +++ /dev/null @@ -1,53 +0,0 @@ -CREATE TABLE libp2p_publish_message_local ON CLUSTER '{cluster}' -( - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String) COMMENT 'Layer of the topic', - topic_fork_digest_value LowCardinality(String) COMMENT 'Fork digest value of the topic', - topic_name LowCardinality(String) COMMENT 'Name of the topic', - topic_encoding LowCardinality(String) COMMENT 'Encoding of the topic', - message_id String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - topic_fork_digest_value, - topic_name, - message_id - ) COMMENT 'Contains the details of the PUBLISH_MESSAGE events from the libp2p client.'; - -CREATE TABLE libp2p_publish_message ON CLUSTER '{cluster}' AS libp2p_publish_message_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_publish_message_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - topic_fork_digest_value, - topic_name, - message_id - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/065_libp2p_trace_duplicate_message.down.sql b/deploy/migrations/clickhouse/065_libp2p_trace_duplicate_message.down.sql deleted file mode 100644 index 15fe1bba..00000000 --- a/deploy/migrations/clickhouse/065_libp2p_trace_duplicate_message.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_duplicate_message ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.libp2p_duplicate_message_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/065_libp2p_trace_duplicate_message.up.sql b/deploy/migrations/clickhouse/065_libp2p_trace_duplicate_message.up.sql deleted file mode 100644 index 01bf043c..00000000 --- a/deploy/migrations/clickhouse/065_libp2p_trace_duplicate_message.up.sql +++ /dev/null @@ -1,61 +0,0 @@ -CREATE TABLE libp2p_duplicate_message_local ON CLUSTER '{cluster}' -( - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event' CODEC(DoubleDelta, ZSTD(1)), - topic_layer LowCardinality(String) COMMENT 'Layer of the topic', - topic_fork_digest_value LowCardinality(String) COMMENT 'Fork digest value of the topic', - topic_name LowCardinality(String) COMMENT 'Name of the topic', - topic_encoding LowCardinality(String) COMMENT 'Encoding of the topic', - seq_number UInt64 COMMENT 'A linearly increasing number that is unique among messages originating from the given peer' CODEC(DoubleDelta, ZSTD(1)), - local_delivery Bool COMMENT 'Indicates if the message was duplicated locally', - peer_id_unique_key Int64 COMMENT 'Unique key for the peer that sent the duplicate message', - message_id String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - message_size UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name, - message_id, - seq_number - ) COMMENT 'Contains the details of the DUPLICATE_MESSAGE events from the libp2p client.'; - -CREATE TABLE libp2p_duplicate_message ON CLUSTER '{cluster}' AS libp2p_duplicate_message_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_duplicate_message_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - topic_fork_digest_value, - topic_name, - message_id, - seq_number - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/066_libp2p_rpc_meta_control_idontwant.down.sql b/deploy/migrations/clickhouse/066_libp2p_rpc_meta_control_idontwant.down.sql deleted file mode 100644 index 6181105a..00000000 --- a/deploy/migrations/clickhouse/066_libp2p_rpc_meta_control_idontwant.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_idontwant ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.libp2p_rpc_meta_control_idontwant_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/066_libp2p_rpc_meta_control_idontwant.up.sql b/deploy/migrations/clickhouse/066_libp2p_rpc_meta_control_idontwant.up.sql deleted file mode 100644 index 69b54d40..00000000 --- a/deploy/migrations/clickhouse/066_libp2p_rpc_meta_control_idontwant.up.sql +++ /dev/null @@ -1,33 +0,0 @@ -CREATE TABLE libp2p_rpc_meta_control_idontwant_local ON CLUSTER '{cluster}' -( - unique_key Int64 COMMENT 'Unique identifier for each IDONTWANT control record', - updated_date_time DateTime COMMENT 'Timestamp when the IDONTWANT control record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the IDONTWANT control event' CODEC(DoubleDelta, ZSTD(1)), - control_index Int32 COMMENT 'Position in the RPC meta control idontwant array' CODEC(DoubleDelta, ZSTD(1)), - message_index Int32 COMMENT 'Position in the RPC meta control idontwant message_ids array' CODEC(DoubleDelta, ZSTD(1)), - rpc_meta_unique_key Int64 COMMENT 'Unique key associated with the IDONTWANT control metadata', - message_id String COMMENT 'Identifier of the message associated with the IDONTWANT control' CODEC(ZSTD(1)), - peer_id_unique_key Int64 COMMENT 'Unique key associated with the identifier of the peer involved in the IDONTWANT control', - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) Engine = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, unique_key, control_index, peer_id_unique_key, message_id, message_index, meta_network_name, meta_client_name) -COMMENT 'Contains the details of the IDONTWANT control messages from the peer.'; - -CREATE TABLE libp2p_rpc_meta_control_idontwant ON CLUSTER '{cluster}' AS libp2p_rpc_meta_control_idontwant_local -ENGINE = Distributed('{cluster}', default, libp2p_rpc_meta_control_idontwant_local, unique_key); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/067_transactions_structlog.down.sql b/deploy/migrations/clickhouse/067_transactions_structlog.down.sql deleted file mode 100644 index 6e1e47cb..00000000 --- a/deploy/migrations/clickhouse/067_transactions_structlog.down.sql +++ /dev/null @@ -1,5 +0,0 @@ -DROP TABLE IF EXISTS admin.execution_block ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS admin.execution_block_local ON CLUSTER '{cluster}' SYNC; - -DROP TABLE IF EXISTS default.canonical_execution_transaction_structlog ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/067_transactions_structlog.up.sql b/deploy/migrations/clickhouse/067_transactions_structlog.up.sql deleted file mode 100644 index 645e9574..00000000 --- a/deploy/migrations/clickhouse/067_transactions_structlog.up.sql +++ /dev/null @@ -1,73 +0,0 @@ -CREATE DATABASE IF NOT EXISTS `admin` ON CLUSTER '{cluster}'; - -CREATE TABLE admin.execution_block_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime(3) CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `processor` LowCardinality(String) COMMENT 'The type of processor that processed the block', - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) -ORDER BY - ( - block_number, - processor, - meta_network_name - ); - -CREATE TABLE admin.execution_block ON CLUSTER '{cluster}' AS admin.execution_block_local ENGINE = Distributed( - '{cluster}', - admin, - execution_block_local, - cityHash64( - block_number, - processor, - meta_network_name - ) -); - -CREATE TABLE default.canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), - `transaction_index` UInt32 COMMENT 'The transaction position in the block' CODEC(DoubleDelta, ZSTD(1)), - `transaction_gas` UInt64 COMMENT 'The transaction gas' CODEC(DoubleDelta, ZSTD(1)), - `transaction_failed` Bool COMMENT 'The transaction failed' CODEC(ZSTD(1)), - `transaction_return_value` Nullable(String) COMMENT 'The transaction return value' CODEC(ZSTD(1)), - `index` UInt32 COMMENT 'The index of this structlog in this transaction' CODEC(DoubleDelta, ZSTD(1)), - `program_counter` UInt32 COMMENT 'The program counter' CODEC(Delta, ZSTD(1)), - `operation` LowCardinality(String) COMMENT 'The operation', - `gas` UInt64 COMMENT 'The gas' CODEC(Delta, ZSTD(1)), - `gas_cost` UInt64 COMMENT 'The gas cost' CODEC(DoubleDelta, ZSTD(1)), - `depth` UInt64 COMMENT 'The depth' CODEC(DoubleDelta, ZSTD(1)), - `return_data` Nullable(String) COMMENT 'The return data' CODEC(ZSTD(1)), - `refund` Nullable(UInt64) COMMENT 'The refund' CODEC(ZSTD(1)), - `error` Nullable(String) COMMENT 'The error' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 201600) -- roughly 1 month of blocks -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - `index` - ) COMMENT 'Contains canonical execution transaction structlog data.'; - -CREATE TABLE default.canonical_execution_transaction_structlog ON CLUSTER '{cluster}' AS default.canonical_execution_transaction_structlog_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_transaction_structlog_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - `index` - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/068_nullable_libp2p_rpc_meta_prune.down.sql b/deploy/migrations/clickhouse/068_nullable_libp2p_rpc_meta_prune.down.sql deleted file mode 100644 index c1662419..00000000 --- a/deploy/migrations/clickhouse/068_nullable_libp2p_rpc_meta_prune.down.sql +++ /dev/null @@ -1,6 +0,0 @@ -ALTER TABLE default.libp2p_rpc_meta_control_prune ON CLUSTER '{cluster}' - MODIFY COLUMN `graft_peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the graft peer involved in the Prune control'; - -ALTER TABLE default.libp2p_rpc_meta_control_prune_local ON CLUSTER '{cluster}' - MODIFY COLUMN `graft_peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the graft peer involved in the Prune control'; - diff --git a/deploy/migrations/clickhouse/068_nullable_libp2p_rpc_meta_prune.up.sql b/deploy/migrations/clickhouse/068_nullable_libp2p_rpc_meta_prune.up.sql deleted file mode 100644 index 2f4c4942..00000000 --- a/deploy/migrations/clickhouse/068_nullable_libp2p_rpc_meta_prune.up.sql +++ /dev/null @@ -1,6 +0,0 @@ -ALTER TABLE default.libp2p_rpc_meta_control_prune ON CLUSTER '{cluster}' - MODIFY COLUMN `graft_peer_id_unique_key` Nullable(Int64) COMMENT 'Unique key associated with the identifier of the graft peer involved in the Prune control'; - -ALTER TABLE default.libp2p_rpc_meta_control_prune_local ON CLUSTER '{cluster}' - MODIFY COLUMN `graft_peer_id_unique_key` Nullable(Int64) COMMENT 'Unique key associated with the identifier of the graft peer involved in the Prune control'; - diff --git a/deploy/migrations/clickhouse/069_transactions_structlog_call_to.down.sql b/deploy/migrations/clickhouse/069_transactions_structlog_call_to.down.sql deleted file mode 100644 index f87b6056..00000000 --- a/deploy/migrations/clickhouse/069_transactions_structlog_call_to.down.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE canonical_execution_transaction_structlog on cluster '{cluster}' - DROP COLUMN call_to_address; - -ALTER TABLE canonical_execution_transaction_structlog_local on cluster '{cluster}' - DROP COLUMN call_to_address; diff --git a/deploy/migrations/clickhouse/069_transactions_structlog_call_to.up.sql b/deploy/migrations/clickhouse/069_transactions_structlog_call_to.up.sql deleted file mode 100644 index 114ef452..00000000 --- a/deploy/migrations/clickhouse/069_transactions_structlog_call_to.up.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - ADD COLUMN call_to_address Nullable(String) COMMENT 'Address of a CALL operation' CODEC(ZSTD(1)) AFTER error; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - ADD COLUMN call_to_address Nullable(String) COMMENT 'Address of a CALL operation' CODEC(ZSTD(1)) AFTER error; diff --git a/deploy/migrations/clickhouse/070_libp2p_gossipsub_aggregate_and_proof.down.sql b/deploy/migrations/clickhouse/070_libp2p_gossipsub_aggregate_and_proof.down.sql deleted file mode 100644 index 4bcbbfe2..00000000 --- a/deploy/migrations/clickhouse/070_libp2p_gossipsub_aggregate_and_proof.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS libp2p_gossipsub_aggregate_and_proof ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS libp2p_gossipsub_aggregate_and_proof_local ON CLUSTER '{cluster}'; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/070_libp2p_gossipsub_aggregate_and_proof.up.sql b/deploy/migrations/clickhouse/070_libp2p_gossipsub_aggregate_and_proof.up.sql deleted file mode 100644 index c4958414..00000000 --- a/deploy/migrations/clickhouse/070_libp2p_gossipsub_aggregate_and_proof.up.sql +++ /dev/null @@ -1,77 +0,0 @@ -CREATE TABLE libp2p_gossipsub_aggregate_and_proof_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - -- ensure the first time this aggregate was seen by a peer is in this table - -- 4294967295 = UInt32 max - version UInt32 DEFAULT 4294967295 - propagation_slot_start_diff COMMENT 'Version of this row, to help with de-duplication we want the latest updated_date_time but lowest propagation_slot_start_diff time' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), - slot UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - wallclock_epoch UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - wallclock_epoch_start_date_time DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - propagation_slot_start_diff UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), - peer_id_unique_key Int64 COMMENT 'Unique key associated with the identifier of the peer', - message_id String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - message_size UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - topic_layer LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', - topic_fork_digest_value LowCardinality(String) COMMENT 'Fork digest value of the topic', - topic_name LowCardinality(String) COMMENT 'Name of the topic', - topic_encoding LowCardinality(String) COMMENT 'Encoding used for the topic', - -- Aggregator specific fields - aggregator_index UInt32 COMMENT 'Index of the validator who created this aggregate' CODEC(DoubleDelta, ZSTD(1)), - -- Embedded attestation fields - committee_index LowCardinality(String) COMMENT 'Committee index from the attestation', - aggregation_bits String COMMENT 'Bitfield of aggregated attestation' CODEC(ZSTD(1)), - beacon_block_root FixedString(66) COMMENT 'Root of the beacon block being attested to' CODEC(ZSTD(1)), - source_epoch UInt32 COMMENT 'Source epoch from the attestation' CODEC(DoubleDelta, ZSTD(1)), - source_root FixedString(66) COMMENT 'Source root from the attestation' CODEC(ZSTD(1)), - target_epoch UInt32 COMMENT 'Target epoch from the attestation' CODEC(DoubleDelta, ZSTD(1)), - target_root FixedString(66) COMMENT 'Target root from the attestation' CODEC(ZSTD(1)), - -- Standard metadata fields - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Network ID associated with the client' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Name of the network associated with the client' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - version -) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY ( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id -) COMMENT 'Table for libp2p gossipsub aggregate and proof data.'; - --- Create distributed table -CREATE TABLE libp2p_gossipsub_aggregate_and_proof ON CLUSTER '{cluster}' AS libp2p_gossipsub_aggregate_and_proof_local -ENGINE = Distributed( - '{cluster}', - default, - libp2p_gossipsub_aggregate_and_proof_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/071_data_column_sidecar.down.sql b/deploy/migrations/clickhouse/071_data_column_sidecar.down.sql deleted file mode 100644 index ecb984c8..00000000 --- a/deploy/migrations/clickhouse/071_data_column_sidecar.down.sql +++ /dev/null @@ -1,3 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_events_data_column_sidecar on cluster '{cluster}'; - -DROP TABLE IF EXISTS beacon_api_eth_v1_events_data_column_sidecar_local on cluster '{cluster}'; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/071_data_column_sidecar.up.sql b/deploy/migrations/clickhouse/071_data_column_sidecar.up.sql deleted file mode 100644 index 14d0f81b..00000000 --- a/deploy/migrations/clickhouse/071_data_column_sidecar.up.sql +++ /dev/null @@ -1,59 +0,0 @@ -CREATE TABLE beacon_api_eth_v1_events_data_column_sidecar_local on cluster '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'The difference between the event_date_time and the slot_start_date_time' CODEC(ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon API event stream payload' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), - `column_index` UInt64 COMMENT 'The index of column in the beacon API event stream payload' CODEC(ZSTD(1)), - `kzg_commitments` Array(FixedString(98)) COMMENT 'The KZG commitments in the beacon API event stream payload' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - column_index - ) COMMENT 'Contains beacon API eventstream "data_column_sidecar" data from each sentry client attached to a beacon node.'; - -CREATE TABLE default.beacon_api_eth_v1_events_data_column_sidecar ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_data_column_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_data_column_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - column_index - ) -); diff --git a/deploy/migrations/clickhouse/072_node_record_execution.down.sql b/deploy/migrations/clickhouse/072_node_record_execution.down.sql deleted file mode 100644 index 3b579e34..00000000 --- a/deploy/migrations/clickhouse/072_node_record_execution.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.node_record_execution ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.node_record_execution_local ON CLUSTER '{cluster}' SYNC; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/072_node_record_execution.up.sql b/deploy/migrations/clickhouse/072_node_record_execution.up.sql deleted file mode 100644 index ee4fe8e1..00000000 --- a/deploy/migrations/clickhouse/072_node_record_execution.up.sql +++ /dev/null @@ -1,71 +0,0 @@ -CREATE TABLE default.node_record_execution_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the event was generated' CODEC(DoubleDelta, ZSTD(1)), - `enr` String COMMENT 'Ethereum Node Record as text' CODEC(ZSTD(1)), - `name` String COMMENT 'Execution client name' CODEC(ZSTD(1)), - `version` LowCardinality(String) COMMENT 'Execution client version' CODEC(ZSTD(1)), - `version_major` LowCardinality(String) COMMENT 'Execution client major version' CODEC(ZSTD(1)), - `version_minor` LowCardinality(String) COMMENT 'Execution client minor version' CODEC(ZSTD(1)), - `version_patch` LowCardinality(String) COMMENT 'Execution client patch version' CODEC(ZSTD(1)), - `implementation` LowCardinality(String) COMMENT 'Execution client implementation' CODEC(ZSTD(1)), - `capabilities` Array(String) COMMENT 'List of capabilities (e.g., eth/65,eth/66)' CODEC(ZSTD(1)), - `protocol_version` String COMMENT 'Protocol version' CODEC(ZSTD(1)), - `total_difficulty` String COMMENT 'Total difficulty of the chain' CODEC(ZSTD(1)), - `head` String COMMENT 'Head block hash' CODEC(ZSTD(1)), - `genesis` String COMMENT 'Genesis block hash' CODEC(ZSTD(1)), - `fork_id_hash` String COMMENT 'Fork ID hash' CODEC(ZSTD(1)), - `fork_id_next` String COMMENT 'Fork ID next block' CODEC(ZSTD(1)), - `node_id` String COMMENT 'Node ID from ENR' CODEC(ZSTD(1)), - `ip` Nullable(IPv6) COMMENT 'IP address of the execution node' CODEC(ZSTD(1)), - `tcp` Nullable(UInt16) COMMENT 'TCP port from ENR' CODEC(DoubleDelta, ZSTD(1)), - `udp` Nullable(UInt16) COMMENT 'UDP port from ENR' CODEC(DoubleDelta, ZSTD(1)), - `has_ipv6` Bool COMMENT 'Whether the execution node has an IPv6 address' CODEC(ZSTD(1)), - `geo_city` LowCardinality(String) COMMENT 'City of the execution node' CODEC(ZSTD(1)), - `geo_country` LowCardinality(String) COMMENT 'Country of the execution node' CODEC(ZSTD(1)), - `geo_country_code` LowCardinality(String) COMMENT 'Country code of the execution node' CODEC(ZSTD(1)), - `geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the execution node' CODEC(ZSTD(1)), - `geo_longitude` Nullable(Float64) COMMENT 'Longitude of the execution node' CODEC(ZSTD(1)), - `geo_latitude` Nullable(Float64) COMMENT 'Latitude of the execution node' CODEC(ZSTD(1)), - `geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the execution node' CODEC(ZSTD(1)), - `geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the execution node' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - node_id, - meta_client_name - ) COMMENT 'Contains execution node records discovered by the Xatu discovery module.'; - -CREATE TABLE default.node_record_execution ON CLUSTER '{cluster}' AS default.node_record_execution_local ENGINE = Distributed( - '{cluster}', - default, - node_record_execution_local, - cityHash64( - event_date_time, - meta_network_name, - node_id, - meta_client_name - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/073_node_record_consensus.down.sql b/deploy/migrations/clickhouse/073_node_record_consensus.down.sql deleted file mode 100644 index 62181755..00000000 --- a/deploy/migrations/clickhouse/073_node_record_consensus.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.node_record_consensus ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.node_record_consensus_local ON CLUSTER '{cluster}'; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/073_node_record_consensus.up.sql b/deploy/migrations/clickhouse/073_node_record_consensus.up.sql deleted file mode 100644 index c72d30fd..00000000 --- a/deploy/migrations/clickhouse/073_node_record_consensus.up.sql +++ /dev/null @@ -1,76 +0,0 @@ -CREATE TABLE default.node_record_consensus_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the discovery module found the node' CODEC(DoubleDelta, ZSTD(1)), - `enr` String COMMENT 'Ethereum Node Record as text' CODEC(ZSTD(1)), - `node_id` Nullable(String) COMMENT 'ID of the Ethereum Node Record' CODEC(ZSTD(1)), - `peer_id_unique_key` Nullable(Int64) COMMENT 'Unique key associated with the identifier of the peer', - `timestamp` Int64 COMMENT 'Event timestamp in unix time' CODEC(DoubleDelta, ZSTD(1)), - `name` String COMMENT 'Consensus client name' CODEC(ZSTD(1)), - `version` LowCardinality(String) COMMENT 'Consensus client version' CODEC(ZSTD(1)), - `version_major` LowCardinality(String) COMMENT 'Consensus client major version' CODEC(ZSTD(1)), - `version_minor` LowCardinality(String) COMMENT 'Consensus client minor version' CODEC(ZSTD(1)), - `version_patch` LowCardinality(String) COMMENT 'Consensus client patch version' CODEC(ZSTD(1)), - `implementation` LowCardinality(String) COMMENT 'Consensus client implementation' CODEC(ZSTD(1)), - `fork_digest` String COMMENT 'Fork digest value' CODEC(ZSTD(1)), - `next_fork_digest` Nullable(String) COMMENT 'Next fork digest of the next scheduled fork' CODEC(ZSTD(1)), - `finalized_root` String COMMENT 'Finalized beacon block root' CODEC(ZSTD(1)), - `finalized_epoch` UInt64 COMMENT 'Finalized epoch number' CODEC(DoubleDelta, ZSTD(1)), - `head_root` String COMMENT 'Head beacon block root' CODEC(ZSTD(1)), - `head_slot` UInt64 COMMENT 'Head slot number' CODEC(DoubleDelta, ZSTD(1)), - `cgc` Nullable(String) COMMENT 'Represents the nodes custody group count' CODEC(ZSTD(1)), - `finalized_epoch_start_date_time` Nullable(DateTime) COMMENT 'Finalized epoch start time' CODEC(DoubleDelta, ZSTD(1)), - `head_slot_start_date_time` Nullable(DateTime) COMMENT 'Head slot start time' CODEC(DoubleDelta, ZSTD(1)), - `ip` Nullable(IPv6) COMMENT 'IP address of the consensus node' CODEC(ZSTD(1)), - `tcp` Nullable(UInt16) COMMENT 'TCP port from ENR' CODEC(DoubleDelta, ZSTD(1)), - `udp` Nullable(UInt16) COMMENT 'UDP port from ENR' CODEC(DoubleDelta, ZSTD(1)), - `quic` Nullable(UInt16) COMMENT 'QUIC port from ENR' CODEC(DoubleDelta, ZSTD(1)), - `has_ipv6` Bool COMMENT 'Whether the consensus node has an IPv6 address' CODEC(ZSTD(1)), - `geo_city` LowCardinality(String) COMMENT 'City of the consensus node' CODEC(ZSTD(1)), - `geo_country` LowCardinality(String) COMMENT 'Country of the consensus node' CODEC(ZSTD(1)), - `geo_country_code` LowCardinality(String) COMMENT 'Country code of the consensus node' CODEC(ZSTD(1)), - `geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the consensus node' CODEC(ZSTD(1)), - `geo_longitude` Nullable(Float64) COMMENT 'Longitude of the consensus node' CODEC(ZSTD(1)), - `geo_latitude` Nullable(Float64) COMMENT 'Latitude of the consensus node' CODEC(ZSTD(1)), - `geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the consensus node' CODEC(ZSTD(1)), - `geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the consensus node' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(event_date_time) -ORDER BY - ( - event_date_time, - meta_network_name, - enr, - meta_client_name - ) COMMENT 'Contains consensus node records discovered by the Xatu discovery module.'; - -CREATE TABLE default.node_record_consensus ON CLUSTER '{cluster}' AS default.node_record_consensus_local ENGINE = Distributed( - '{cluster}', - default, - node_record_consensus_local, - cityHash64( - event_date_time, - meta_network_name, - enr, - meta_client_name - ) -); diff --git a/deploy/migrations/clickhouse/074_libp2p_gossipsub_data_column_sidecar.down.sql b/deploy/migrations/clickhouse/074_libp2p_gossipsub_data_column_sidecar.down.sql deleted file mode 100644 index 091ecdbe..00000000 --- a/deploy/migrations/clickhouse/074_libp2p_gossipsub_data_column_sidecar.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_gossipsub_data_column_sidecar ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_gossipsub_data_column_sidecar_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/074_libp2p_gossipsub_data_column_sidecar.up.sql b/deploy/migrations/clickhouse/074_libp2p_gossipsub_data_column_sidecar.up.sql deleted file mode 100644 index d7c7a018..00000000 --- a/deploy/migrations/clickhouse/074_libp2p_gossipsub_data_column_sidecar.up.sql +++ /dev/null @@ -1,68 +0,0 @@ -CREATE TABLE libp2p_gossipsub_data_column_sidecar_local ON CLUSTER '{cluster}' -( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `version` UInt32 DEFAULT 4294967295 - propagation_slot_start_diff COMMENT 'Version of this row, to help with de-duplication we want the latest updated_date_time but lowest propagation_slot_start_diff time' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'Timestamp of the event with millisecond precision' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'Start date and time of the slot' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'Epoch number associated with the event' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'Start date and time of the epoch' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot` UInt32 COMMENT 'Slot number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_slot_start_date_time` DateTime COMMENT 'Start date and time of the wall clock slot when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch` UInt32 COMMENT 'Epoch number of the wall clock when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `wallclock_epoch_start_date_time` DateTime COMMENT 'Start date and time of the wall clock epoch when the event was received' CODEC(DoubleDelta, ZSTD(1)), - `propagation_slot_start_diff` UInt32 COMMENT 'Difference in slot start time for propagation' CODEC(ZSTD(1)), - `proposer_index` UInt32 COMMENT 'The proposer index of the beacon block' CODEC(ZSTD(1)), - `column_index` UInt64 COMMENT 'Column index associated with the record' CODEC(ZSTD(1)), - `kzg_commitments_count` UInt32 COMMENT 'Number of KZG commitments associated with the record' CODEC(ZSTD(1)), - `parent_root` FixedString(66) COMMENT 'Parent root of the beacon block' CODEC(ZSTD(1)), - `state_root` FixedString(66) COMMENT 'State root of the beacon block' CODEC(ZSTD(1)), - `peer_id_unique_key` Int64 COMMENT 'Unique key associated with the identifier of the peer', - `message_id` String COMMENT 'Identifier of the message' CODEC(ZSTD(1)), - `message_size` UInt32 COMMENT 'Size of the message in bytes' CODEC(ZSTD(1)), - `topic_layer` LowCardinality(String) COMMENT 'Layer of the topic in the gossipsub protocol', - `topic_fork_digest_value` LowCardinality(String) COMMENT 'Fork digest value of the topic', - `topic_name` LowCardinality(String) COMMENT 'Name of the topic', - `topic_encoding` LowCardinality(String) COMMENT 'Encoding used for the topic', - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Network ID associated with the client' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Name of the network associated with the client' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toYYYYMM(event_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) COMMENT 'Table for libp2p gossipsub data column sidecar data'; - -CREATE TABLE libp2p_gossipsub_data_column_sidecar ON CLUSTER '{cluster}' AS libp2p_gossipsub_data_column_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - libp2p_gossipsub_data_column_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - message_id - ) -); diff --git a/deploy/migrations/clickhouse/075_libp2p_synthetic_heartbeat.down.sql b/deploy/migrations/clickhouse/075_libp2p_synthetic_heartbeat.down.sql deleted file mode 100644 index 87cb3c80..00000000 --- a/deploy/migrations/clickhouse/075_libp2p_synthetic_heartbeat.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.libp2p_synthetic_heartbeat ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.libp2p_synthetic_heartbeat_local ON CLUSTER '{cluster}'; \ No newline at end of file diff --git a/deploy/migrations/clickhouse/075_libp2p_synthetic_heartbeat.up.sql b/deploy/migrations/clickhouse/075_libp2p_synthetic_heartbeat.up.sql deleted file mode 100644 index 98222244..00000000 --- a/deploy/migrations/clickhouse/075_libp2p_synthetic_heartbeat.up.sql +++ /dev/null @@ -1,113 +0,0 @@ --- Creating local and distributed tables for libp2p_synthetic_heartbeat -CREATE TABLE libp2p_synthetic_heartbeat_local ON CLUSTER '{cluster}' -( - updated_date_time DateTime CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - remote_peer_id_unique_key Int64, - remote_maddrs String CODEC(ZSTD(1)), - latency_ms Nullable(Int64) CODEC(ZSTD(1)), - direction LowCardinality(String), - protocols Array(String) CODEC(ZSTD(1)), - connection_age_ms Nullable(Int64) CODEC(ZSTD(1)), - remote_agent_implementation LowCardinality(String), - remote_agent_version LowCardinality(String), - remote_agent_version_major LowCardinality(String), - remote_agent_version_minor LowCardinality(String), - remote_agent_version_patch LowCardinality(String), - remote_agent_platform LowCardinality(String), - remote_ip Nullable(IPv6) CODEC(ZSTD(1)), - remote_port Nullable(UInt16) CODEC(ZSTD(1)), - remote_geo_city LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_country LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - remote_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - remote_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - remote_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - remote_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) -) Engine = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY ( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - updated_date_time -); - -ALTER TABLE libp2p_synthetic_heartbeat_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains heartbeat events from libp2p peers', -COMMENT COLUMN updated_date_time 'Timestamp when the record was last updated', -COMMENT COLUMN event_date_time 'Timestamp of the heartbeat event', -COMMENT COLUMN remote_peer_id_unique_key 'Unique key of the remote peer', -COMMENT COLUMN remote_maddrs 'Multiaddress of the remote peer', -COMMENT COLUMN latency_ms 'EWMA latency in milliseconds (0 if unavailable)', -COMMENT COLUMN direction 'Connection direction (Unknown/Inbound/Outbound)', -COMMENT COLUMN protocols 'List of supported protocols', -COMMENT COLUMN connection_age_ms 'Connection age in milliseconds', -COMMENT COLUMN remote_agent_implementation 'Implementation of the remote peer', -COMMENT COLUMN remote_agent_version 'Version of the remote peer', -COMMENT COLUMN remote_agent_version_major 'Major version of the remote peer', -COMMENT COLUMN remote_agent_version_minor 'Minor version of the remote peer', -COMMENT COLUMN remote_agent_version_patch 'Patch version of the remote peer', -COMMENT COLUMN remote_agent_platform 'Platform of the remote peer', -COMMENT COLUMN remote_ip 'IP address of the remote peer', -COMMENT COLUMN remote_port 'Port of the remote peer', -COMMENT COLUMN remote_geo_city 'City of the remote peer', -COMMENT COLUMN remote_geo_country 'Country of the remote peer', -COMMENT COLUMN remote_geo_country_code 'Country code of the remote peer', -COMMENT COLUMN remote_geo_continent_code 'Continent code of the remote peer', -COMMENT COLUMN remote_geo_longitude 'Longitude of the remote peer', -COMMENT COLUMN remote_geo_latitude 'Latitude of the remote peer', -COMMENT COLUMN remote_geo_autonomous_system_number 'ASN of the remote peer', -COMMENT COLUMN remote_geo_autonomous_system_organization 'AS organization of the remote peer', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client', -COMMENT COLUMN meta_client_version 'Version of the client', -COMMENT COLUMN meta_client_implementation 'Implementation of the client', -COMMENT COLUMN meta_client_os 'Operating system of the client', -COMMENT COLUMN meta_client_ip 'IP address of the client', -COMMENT COLUMN meta_client_geo_city 'City of the client', -COMMENT COLUMN meta_client_geo_country 'Country of the client', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'ASN of the client', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'AS organization of the client', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name'; - -CREATE TABLE libp2p_synthetic_heartbeat ON CLUSTER '{cluster}' AS libp2p_synthetic_heartbeat_local -ENGINE = Distributed( - '{cluster}', - default, - libp2p_synthetic_heartbeat_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - remote_peer_id_unique_key, - updated_date_time - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/076_block_root_libp2p.down.sql b/deploy/migrations/clickhouse/076_block_root_libp2p.down.sql deleted file mode 100644 index 1f931b6e..00000000 --- a/deploy/migrations/clickhouse/076_block_root_libp2p.down.sql +++ /dev/null @@ -1,11 +0,0 @@ -ALTER TABLE default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' -DROP COLUMN beacon_block_root; - -ALTER TABLE default.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' -DROP COLUMN beacon_block_root; - -ALTER TABLE default.libp2p_gossipsub_data_column_sidecar ON CLUSTER '{cluster}' -DROP COLUMN beacon_block_root; - -ALTER TABLE default.libp2p_gossipsub_data_column_sidecar_local ON CLUSTER '{cluster}' -DROP COLUMN beacon_block_root; diff --git a/deploy/migrations/clickhouse/076_block_root_libp2p.up.sql b/deploy/migrations/clickhouse/076_block_root_libp2p.up.sql deleted file mode 100644 index 410225e1..00000000 --- a/deploy/migrations/clickhouse/076_block_root_libp2p.up.sql +++ /dev/null @@ -1,12 +0,0 @@ - -ALTER TABLE default.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' -ADD COLUMN beacon_block_root FixedString(66) Codec(ZSTD(1)) AFTER blob_index; - -ALTER TABLE default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' -ADD COLUMN beacon_block_root FixedString(66) Codec(ZSTD(1)) AFTER blob_index; - -ALTER TABLE default.libp2p_gossipsub_data_column_sidecar_local ON CLUSTER '{cluster}' -ADD COLUMN beacon_block_root FixedString(66) Codec(ZSTD(1)) AFTER kzg_commitments_count; - -ALTER TABLE default.libp2p_gossipsub_data_column_sidecar ON CLUSTER '{cluster}' -ADD COLUMN beacon_block_root FixedString(66) Codec(ZSTD(1)) AFTER kzg_commitments_count; diff --git a/deploy/migrations/clickhouse/077_data_column_sidecar_kzg_commitments_count.down.sql b/deploy/migrations/clickhouse/077_data_column_sidecar_kzg_commitments_count.down.sql deleted file mode 100644 index 9ed2a92c..00000000 --- a/deploy/migrations/clickhouse/077_data_column_sidecar_kzg_commitments_count.down.sql +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_events_data_column_sidecar ON CLUSTER '{cluster}'; - -ALTER TABLE beacon_api_eth_v1_events_data_column_sidecar_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS kzg_commitments_count; - -ALTER TABLE beacon_api_eth_v1_events_data_column_sidecar_local ON CLUSTER '{cluster}' - ADD COLUMN kzg_commitments Array(FixedString(98)) COMMENT 'The KZG commitments in the beacon API event stream payload' CODEC(ZSTD(1)) AFTER column_index; - -CREATE TABLE default.beacon_api_eth_v1_events_data_column_sidecar ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_data_column_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_data_column_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - column_index - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/077_data_column_sidecar_kzg_commitments_count.up.sql b/deploy/migrations/clickhouse/077_data_column_sidecar_kzg_commitments_count.up.sql deleted file mode 100644 index 9d0e32e9..00000000 --- a/deploy/migrations/clickhouse/077_data_column_sidecar_kzg_commitments_count.up.sql +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_events_data_column_sidecar ON CLUSTER '{cluster}' SYNC; - -ALTER TABLE beacon_api_eth_v1_events_data_column_sidecar_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS kzg_commitments; - -ALTER TABLE beacon_api_eth_v1_events_data_column_sidecar_local ON CLUSTER '{cluster}' - ADD COLUMN kzg_commitments_count UInt32 COMMENT 'Number of KZG commitments associated with the record' CODEC(ZSTD(1)) AFTER column_index; - -CREATE TABLE default.beacon_api_eth_v1_events_data_column_sidecar ON CLUSTER '{cluster}' AS default.beacon_api_eth_v1_events_data_column_sidecar_local ENGINE = Distributed( - '{cluster}', - default, - beacon_api_eth_v1_events_data_column_sidecar_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - column_index - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/078_add_peerdas_rpc_columns.down.sql b/deploy/migrations/clickhouse/078_add_peerdas_rpc_columns.down.sql deleted file mode 100644 index 045d32f6..00000000 --- a/deploy/migrations/clickhouse/078_add_peerdas_rpc_columns.down.sql +++ /dev/null @@ -1,48 +0,0 @@ --- Remove Direction column from distributed tables first -DROP TABLE IF EXISTS libp2p_handle_status ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS libp2p_handle_metadata ON CLUSTER '{cluster}' SYNC; - --- Remove Direction column from local tables -ALTER TABLE libp2p_handle_status_local ON CLUSTER '{cluster}' -DROP COLUMN IF EXISTS direction; - -ALTER TABLE libp2p_handle_status_local ON CLUSTER '{cluster}' -DROP COLUMN IF EXISTS request_earliest_available_slot; - -ALTER TABLE libp2p_handle_status_local ON CLUSTER '{cluster}' -DROP COLUMN IF EXISTS response_earliest_available_slot; - -ALTER TABLE libp2p_handle_metadata_local ON CLUSTER '{cluster}' -DROP COLUMN IF EXISTS direction; - -ALTER TABLE libp2p_handle_metadata_local ON CLUSTER '{cluster}' -DROP COLUMN IF EXISTS custody_group_count; - --- Recreate distributed tables -CREATE TABLE libp2p_handle_status ON CLUSTER '{cluster}' AS libp2p_handle_status_local -ENGINE = Distributed( - '{cluster}', - default, - libp2p_handle_status_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - latency_milliseconds - ) -); - -CREATE TABLE libp2p_handle_metadata ON CLUSTER '{cluster}' AS libp2p_handle_metadata_local -ENGINE = Distributed( - '{cluster}', - default, - libp2p_handle_metadata_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - latency_milliseconds - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/078_add_peerdas_rpc_columns.up.sql b/deploy/migrations/clickhouse/078_add_peerdas_rpc_columns.up.sql deleted file mode 100644 index ba159a92..00000000 --- a/deploy/migrations/clickhouse/078_add_peerdas_rpc_columns.up.sql +++ /dev/null @@ -1,57 +0,0 @@ --- Drop distributed tables -DROP TABLE IF EXISTS libp2p_handle_status ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS libp2p_handle_metadata ON CLUSTER '{cluster}' SYNC; - --- Add Direction column to libp2p_handle_status_local -ALTER TABLE libp2p_handle_status_local ON CLUSTER '{cluster}' -ADD COLUMN IF NOT EXISTS direction LowCardinality(Nullable(String)) COMMENT 'Direction of the RPC request (inbound or outbound)' CODEC(ZSTD(1)) -AFTER protocol; - --- Add request_earliest_available_slot column to libp2p_handle_status_local -ALTER TABLE libp2p_handle_status_local ON CLUSTER '{cluster}' -ADD COLUMN IF NOT EXISTS request_earliest_available_slot Nullable(UInt32) COMMENT 'Requested earliest available slot' CODEC(ZSTD(1)) -AFTER request_head_slot; - --- Add response_earliest_available_slot column to libp2p_handle_status_local -ALTER TABLE libp2p_handle_status_local ON CLUSTER '{cluster}' -ADD COLUMN IF NOT EXISTS response_earliest_available_slot Nullable(UInt32) COMMENT 'Response earliest available slot' CODEC(ZSTD(1)) -AFTER response_head_slot; - --- Add Direction column to libp2p_handle_metadata_local -ALTER TABLE libp2p_handle_metadata_local ON CLUSTER '{cluster}' -ADD COLUMN IF NOT EXISTS direction LowCardinality(Nullable(String)) COMMENT 'Direction of the RPC request (inbound or outbound)' CODEC(ZSTD(1)) -AFTER protocol; - --- Add custody_group_count column to libp2p_handle_metadata_local -ALTER TABLE libp2p_handle_metadata_local ON CLUSTER '{cluster}' -ADD COLUMN IF NOT EXISTS custody_group_count Nullable(UInt8) COMMENT 'Number of custody groups (0-127)' CODEC(ZSTD(1)) -AFTER syncnets; - --- Recreate distributed tables -CREATE TABLE libp2p_handle_status ON CLUSTER '{cluster}' AS libp2p_handle_status_local -ENGINE = Distributed( - '{cluster}', - default, - libp2p_handle_status_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - latency_milliseconds - ) -); - -CREATE TABLE libp2p_handle_metadata ON CLUSTER '{cluster}' AS libp2p_handle_metadata_local -ENGINE = Distributed( - '{cluster}', - default, - libp2p_handle_metadata_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - latency_milliseconds - ) -); \ No newline at end of file diff --git a/deploy/migrations/clickhouse/079_custody_probe.down.sql b/deploy/migrations/clickhouse/079_custody_probe.down.sql deleted file mode 100644 index f1ca4ef6..00000000 --- a/deploy/migrations/clickhouse/079_custody_probe.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS libp2p_rpc_data_column_custody_probe ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS libp2p_rpc_data_column_custody_probe_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/079_custody_probe.up.sql b/deploy/migrations/clickhouse/079_custody_probe.up.sql deleted file mode 100644 index b2b77955..00000000 --- a/deploy/migrations/clickhouse/079_custody_probe.up.sql +++ /dev/null @@ -1,80 +0,0 @@ --- Create local table (actual storage) -CREATE TABLE IF NOT EXISTS libp2p_rpc_data_column_custody_probe_local ON CLUSTER '{cluster}' ( - -- Timestamps - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'When the probe was executed' CODEC(DoubleDelta, ZSTD(1)), - - -- Probe identifiers - slot UInt32 COMMENT 'Slot number being probed' CODEC(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - epoch UInt32 COMMENT 'Epoch number of the slot being probed' CODEC(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - - wallclock_request_slot UInt32 COMMENT 'The wallclock slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - wallclock_request_slot_start_date_time DateTime COMMENT 'The start time for the slot when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - wallclock_request_epoch UInt32 COMMENT 'The wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - wallclock_request_epoch_start_date_time DateTime COMMENT 'The start time for the wallclock epoch when the request was sent' CODEC(DoubleDelta, ZSTD(1)), - - -- Column information - column_index UInt64 COMMENT 'Column index being probed' CODEC(ZSTD(1)), - column_rows_count UInt16 COMMENT 'Number of rows in the column' CODEC(ZSTD(1)), - beacon_block_root FixedString(66) COMMENT 'Root of the beacon block' CODEC(ZSTD(1)), - - -- Peer information - peer_id_unique_key Int64 COMMENT 'Unique key associated with the identifier of the peer', - - -- Probe results - result LowCardinality(String) COMMENT 'Result of the probe' CODEC(ZSTD(1)), - response_time_ms Int32 COMMENT 'Response time in milliseconds' CODEC(ZSTD(1)), - error Nullable(String) COMMENT 'Error message if probe failed' CODEC(ZSTD(1)), - - -- Standard metadata fields - meta_client_name LowCardinality(String) COMMENT 'Name of the client that executed the probe', - meta_client_id String COMMENT 'Unique Session ID of the client' CODEC(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client' CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client' CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client' CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client' CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client' CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client' CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client' CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client' CODEC(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name', - meta_labels Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(event_date_time) -ORDER BY ( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - slot, - column_index -) -COMMENT 'Contains custody probe events for data column availability verification'; - --- Create distributed table (query interface) -CREATE TABLE IF NOT EXISTS libp2p_rpc_data_column_custody_probe ON CLUSTER '{cluster}' -AS libp2p_rpc_data_column_custody_probe_local -ENGINE = Distributed( - '{cluster}', - default, - libp2p_rpc_data_column_custody_probe_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name, - peer_id_unique_key, - slot, - column_index - ) -); diff --git a/deploy/migrations/clickhouse/080_destroy_dbt.down.sql b/deploy/migrations/clickhouse/080_destroy_dbt.down.sql deleted file mode 100644 index e229b8c3..00000000 --- a/deploy/migrations/clickhouse/080_destroy_dbt.down.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE DATABASE IF NOT EXISTS dbt ON CLUSTER '{cluster}' diff --git a/deploy/migrations/clickhouse/080_destroy_dbt.up.sql b/deploy/migrations/clickhouse/080_destroy_dbt.up.sql deleted file mode 100644 index 13b5f5e9..00000000 --- a/deploy/migrations/clickhouse/080_destroy_dbt.up.sql +++ /dev/null @@ -1 +0,0 @@ -DROP DATABASE dbt ON CLUSTER '{cluster}' SYNC diff --git a/deploy/migrations/clickhouse/081_execution_state_size.down.sql b/deploy/migrations/clickhouse/081_execution_state_size.down.sql deleted file mode 100644 index 8f1f5cde..00000000 --- a/deploy/migrations/clickhouse/081_execution_state_size.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS execution_state_size ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS execution_state_size_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/081_execution_state_size.up.sql b/deploy/migrations/clickhouse/081_execution_state_size.up.sql deleted file mode 100644 index 29a56f0a..00000000 --- a/deploy/migrations/clickhouse/081_execution_state_size.up.sql +++ /dev/null @@ -1,65 +0,0 @@ -CREATE TABLE execution_state_size_local ON CLUSTER '{cluster}' ( - -- Timestamps - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' Codec(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'When the state size measurement was taken' Codec(DoubleDelta, ZSTD(1)), - - -- Block information - block_number UInt64 COMMENT 'Block number at which the state size was measured' Codec(DoubleDelta, ZSTD(1)), - state_root FixedString(66) COMMENT 'State root hash of the execution layer at this block' Codec(ZSTD(1)), - - -- Account state size metrics - accounts UInt64 COMMENT 'Total number of accounts in the state' Codec(ZSTD(1)), - account_bytes UInt64 COMMENT 'Total bytes used by account data' Codec(ZSTD(1)), - account_trienodes UInt64 COMMENT 'Number of trie nodes in the account trie' Codec(ZSTD(1)), - account_trienode_bytes UInt64 COMMENT 'Total bytes used by account trie nodes' Codec(ZSTD(1)), - - -- Contract code size metrics - contract_codes UInt64 COMMENT 'Total number of contract codes stored' Codec(ZSTD(1)), - contract_code_bytes UInt64 COMMENT 'Total bytes used by contract code' Codec(ZSTD(1)), - - -- Storage size metrics - storages UInt64 COMMENT 'Total number of storage slots in the state' Codec(ZSTD(1)), - storage_bytes UInt64 COMMENT 'Total bytes used by storage data' Codec(ZSTD(1)), - storage_trienodes UInt64 COMMENT 'Number of trie nodes in the storage trie' Codec(ZSTD(1)), - storage_trienode_bytes UInt64 COMMENT 'Total bytes used by storage trie nodes' Codec(ZSTD(1)), - - -- Standard metadata fields - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' Codec(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' Codec(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name', - meta_execution_version LowCardinality(String) COMMENT 'Execution client version that generated the event', - meta_execution_version_major LowCardinality(String) COMMENT 'Execution client major version that generated the event', - meta_execution_version_minor LowCardinality(String) COMMENT 'Execution client minor version that generated the event', - meta_execution_version_patch LowCardinality(String) COMMENT 'Execution client patch version that generated the event', - meta_execution_implementation LowCardinality(String) COMMENT 'Execution client implementation that generated the event', - meta_labels Map(String, String) COMMENT 'Labels associated with the event' Codec(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY intDiv(block_number, 5000000) -ORDER BY (block_number, meta_network_name, meta_client_name, state_root, event_date_time) COMMENT 'Contains execution layer state size metrics including account, contract code, and storage data measurements at specific block heights.'; - -CREATE TABLE execution_state_size ON CLUSTER '{cluster}' AS execution_state_size_local -ENGINE = Distributed( - '{cluster}', - default, - execution_state_size_local, - cityHash64( - block_number, - meta_network_name, - meta_client_name, - state_root, - event_date_time - ) -); diff --git a/deploy/migrations/clickhouse/082_execution_transaction.down.sql b/deploy/migrations/clickhouse/082_execution_transaction.down.sql deleted file mode 100644 index fca1743e..00000000 --- a/deploy/migrations/clickhouse/082_execution_transaction.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS execution_transaction ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS execution_transaction_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/082_execution_transaction.up.sql b/deploy/migrations/clickhouse/082_execution_transaction.up.sql deleted file mode 100644 index 41e6b77e..00000000 --- a/deploy/migrations/clickhouse/082_execution_transaction.up.sql +++ /dev/null @@ -1,51 +0,0 @@ -CREATE TABLE default.execution_transaction_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `block_hash` FixedString(66) COMMENT 'The block hash' CODEC(ZSTD(1)), - `parent_hash` FixedString(66) COMMENT 'The parent block hash' CODEC(ZSTD(1)), - `position` UInt32 COMMENT 'The position of the transaction in the beacon block' CODEC(DoubleDelta, ZSTD(1)), - `hash` FixedString(66) COMMENT 'The hash of the transaction' CODEC(ZSTD(1)), - `from` FixedString(42) COMMENT 'The address of the account that sent the transaction' CODEC(ZSTD(1)), - `to` Nullable(FixedString(42)) COMMENT 'The address of the account that is the transaction recipient' CODEC(ZSTD(1)), - `nonce` UInt64 COMMENT 'The nonce of the sender account at the time of the transaction' CODEC(ZSTD(1)), - `gas_price` UInt128 COMMENT 'The gas price of the transaction in wei' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'The maximum gas provided for the transaction execution' CODEC(ZSTD(1)), - `gas_tip_cap` Nullable(UInt128) COMMENT 'The priority fee (tip) the user has set for the transaction' CODEC(ZSTD(1)), - `gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), - `value` UInt128 COMMENT 'The value transferred with the transaction in wei' CODEC(ZSTD(1)), - `type` UInt8 COMMENT 'The type of the transaction' CODEC(ZSTD(1)), - `size` UInt32 COMMENT 'The size of the transaction data in bytes' CODEC(ZSTD(1)), - `call_data_size` UInt32 COMMENT 'The size of the call data of the transaction in bytes' CODEC(ZSTD(1)), - `blob_gas` Nullable(UInt64) COMMENT 'The maximum gas provided for the blob transaction execution' CODEC(ZSTD(1)), - `blob_gas_fee_cap` Nullable(UInt128) COMMENT 'The max fee the user has set for the transaction' CODEC(ZSTD(1)), - `blob_hashes` Array(String) COMMENT 'The hashes of the blob commitments for blob transactions' CODEC(ZSTD(1)), - `success` Bool COMMENT 'The transaction success' CODEC(ZSTD(1)), - `n_input_bytes` UInt32 COMMENT 'The transaction input bytes' CODEC(ZSTD(1)), - `n_input_zero_bytes` UInt32 COMMENT 'The transaction input zero bytes' CODEC(ZSTD(1)), - `n_input_nonzero_bytes` UInt32 COMMENT 'The transaction input nonzero bytes' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 5000000) -ORDER BY - ( - block_number, - meta_network_name, - block_hash, - position - ) COMMENT 'Contains execution transaction data that may not be canonical.'; - -CREATE TABLE default.execution_transaction ON CLUSTER '{cluster}' AS default.execution_transaction_local ENGINE = Distributed( - '{cluster}', - default, - execution_transaction_local, - cityHash64( - block_number, - meta_network_name, - block_hash, - position - ) -); diff --git a/deploy/migrations/clickhouse/083_blob_submitter.down.sql b/deploy/migrations/clickhouse/083_blob_submitter.down.sql deleted file mode 100644 index e8a14461..00000000 --- a/deploy/migrations/clickhouse/083_blob_submitter.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.blob_submitter ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.blob_submitter_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/083_blob_submitter.up.sql b/deploy/migrations/clickhouse/083_blob_submitter.up.sql deleted file mode 100644 index 97c55488..00000000 --- a/deploy/migrations/clickhouse/083_blob_submitter.up.sql +++ /dev/null @@ -1,25 +0,0 @@ -CREATE TABLE default.blob_submitter_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `address` FixedString(66) COMMENT 'Ethereum address of the blob submitter' CODEC(ZSTD(1)), - `name` String COMMENT 'Name of the blob submitter' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) -ORDER BY - ( - address, - meta_network_name - ) COMMENT 'Contains blob submitter address to name mappings.'; - -CREATE TABLE default.blob_submitter ON CLUSTER '{cluster}' AS default.blob_submitter_local ENGINE = Distributed( - '{cluster}', - default, - blob_submitter_local, - cityHash64( - address, - meta_network_name - ) -); diff --git a/deploy/migrations/clickhouse/084_consensus_engine_api_new_payload.down.sql b/deploy/migrations/clickhouse/084_consensus_engine_api_new_payload.down.sql deleted file mode 100644 index 2998d885..00000000 --- a/deploy/migrations/clickhouse/084_consensus_engine_api_new_payload.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS consensus_engine_api_new_payload ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS consensus_engine_api_new_payload_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/084_consensus_engine_api_new_payload.up.sql b/deploy/migrations/clickhouse/084_consensus_engine_api_new_payload.up.sql deleted file mode 100644 index 2d80bf63..00000000 --- a/deploy/migrations/clickhouse/084_consensus_engine_api_new_payload.up.sql +++ /dev/null @@ -1,69 +0,0 @@ -CREATE TABLE consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' ( - -- Timestamps - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' Codec(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'When the sentry received the event' Codec(DoubleDelta, ZSTD(1)), - requested_date_time DateTime64(3) COMMENT 'Timestamp when the engine_newPayload call was initiated' Codec(DoubleDelta, ZSTD(1)), - - -- Timing - duration_ms UInt64 COMMENT 'How long the engine_newPayload call took in milliseconds' Codec(ZSTD(1)), - - -- Beacon context - slot UInt32 COMMENT 'Slot number of the beacon block containing the payload' Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime COMMENT 'The wall clock time when the slot started' Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 COMMENT 'Epoch number derived from the slot' Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime COMMENT 'The wall clock time when the epoch started' Codec(DoubleDelta, ZSTD(1)), - block_root FixedString(66) COMMENT 'Root of the beacon block (hex encoded with 0x prefix)' Codec(ZSTD(1)), - parent_block_root FixedString(66) COMMENT 'Root of the parent beacon block (hex encoded with 0x prefix)' Codec(ZSTD(1)), - proposer_index UInt32 COMMENT 'Validator index of the block proposer' Codec(ZSTD(1)), - - -- Execution payload details - block_number UInt64 COMMENT 'Execution block number' Codec(DoubleDelta, ZSTD(1)), - block_hash FixedString(66) COMMENT 'Execution block hash (hex encoded with 0x prefix)' Codec(ZSTD(1)), - parent_hash FixedString(66) COMMENT 'Parent execution block hash (hex encoded with 0x prefix)' Codec(ZSTD(1)), - gas_used UInt64 COMMENT 'Total gas used by all transactions in the block' Codec(ZSTD(1)), - gas_limit UInt64 COMMENT 'Gas limit of the block' Codec(ZSTD(1)), - tx_count UInt32 COMMENT 'Number of transactions in the block' Codec(ZSTD(1)), - blob_count UInt32 COMMENT 'Number of blobs in the block' Codec(ZSTD(1)), - - -- Response from EL - status LowCardinality(String) COMMENT 'Payload status returned by EL (VALID, INVALID, SYNCING, ACCEPTED, INVALID_BLOCK_HASH)', - latest_valid_hash Nullable(FixedString(66)) COMMENT 'Latest valid hash when status is INVALID (hex encoded with 0x prefix)' Codec(ZSTD(1)), - validation_error Nullable(String) COMMENT 'Error message when validation fails' Codec(ZSTD(1)), - - -- Meta - method_version LowCardinality(String) COMMENT 'Version of the engine_newPayload method (e.g., V3, V4)', - - -- Standard metadata fields - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' Codec(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' Codec(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name, block_hash, event_date_time) COMMENT 'Contains timing and instrumentation data for engine_newPayload calls between the consensus and execution layer.'; - -CREATE TABLE consensus_engine_api_new_payload ON CLUSTER '{cluster}' AS consensus_engine_api_new_payload_local -ENGINE = Distributed( - '{cluster}', - default, - consensus_engine_api_new_payload_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_hash, - event_date_time - ) -); diff --git a/deploy/migrations/clickhouse/085_consensus_engine_api_new_payload_execution_version.down.sql b/deploy/migrations/clickhouse/085_consensus_engine_api_new_payload_execution_version.down.sql deleted file mode 100644 index 1b518f09..00000000 --- a/deploy/migrations/clickhouse/085_consensus_engine_api_new_payload_execution_version.down.sql +++ /dev/null @@ -1,31 +0,0 @@ --- Remove execution client version columns from consensus_engine_api_new_payload table - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_version_patch; - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_version_minor; - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_version_major; - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_implementation; - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_version; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_version_patch; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_version_minor; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_version_major; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_implementation; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -DROP COLUMN meta_execution_version; diff --git a/deploy/migrations/clickhouse/085_consensus_engine_api_new_payload_execution_version.up.sql b/deploy/migrations/clickhouse/085_consensus_engine_api_new_payload_execution_version.up.sql deleted file mode 100644 index 64803d9b..00000000 --- a/deploy/migrations/clickhouse/085_consensus_engine_api_new_payload_execution_version.up.sql +++ /dev/null @@ -1,32 +0,0 @@ --- Add execution client version columns to consensus_engine_api_new_payload table - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_version LowCardinality(String) DEFAULT '' COMMENT 'Full execution client version string from web3_clientVersion RPC' AFTER method_version; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_implementation LowCardinality(String) DEFAULT '' COMMENT 'Execution client implementation name (e.g., Geth, Nethermind, Besu, Reth, Erigon)' AFTER meta_execution_version; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_version_major LowCardinality(String) DEFAULT '' COMMENT 'Execution client major version number' AFTER meta_execution_implementation; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_version_minor LowCardinality(String) DEFAULT '' COMMENT 'Execution client minor version number' AFTER meta_execution_version_major; - -ALTER TABLE default.consensus_engine_api_new_payload_local ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_version_patch LowCardinality(String) DEFAULT '' COMMENT 'Execution client patch version number' AFTER meta_execution_version_minor; - --- Add to distributed table -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_version LowCardinality(String) DEFAULT '' COMMENT 'Full execution client version string from web3_clientVersion RPC' AFTER method_version; - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_implementation LowCardinality(String) DEFAULT '' COMMENT 'Execution client implementation name (e.g., Geth, Nethermind, Besu, Reth, Erigon)' AFTER meta_execution_version; - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_version_major LowCardinality(String) DEFAULT '' COMMENT 'Execution client major version number' AFTER meta_execution_implementation; - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_version_minor LowCardinality(String) DEFAULT '' COMMENT 'Execution client minor version number' AFTER meta_execution_version_major; - -ALTER TABLE default.consensus_engine_api_new_payload ON CLUSTER '{cluster}' -ADD COLUMN meta_execution_version_patch LowCardinality(String) DEFAULT '' COMMENT 'Execution client patch version number' AFTER meta_execution_version_minor; diff --git a/deploy/migrations/clickhouse/086_consensus_engine_api_get_blobs.down.sql b/deploy/migrations/clickhouse/086_consensus_engine_api_get_blobs.down.sql deleted file mode 100644 index 7090963a..00000000 --- a/deploy/migrations/clickhouse/086_consensus_engine_api_get_blobs.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS consensus_engine_api_get_blobs ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS consensus_engine_api_get_blobs_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/086_consensus_engine_api_get_blobs.up.sql b/deploy/migrations/clickhouse/086_consensus_engine_api_get_blobs.up.sql deleted file mode 100644 index efc831fa..00000000 --- a/deploy/migrations/clickhouse/086_consensus_engine_api_get_blobs.up.sql +++ /dev/null @@ -1,70 +0,0 @@ -CREATE TABLE consensus_engine_api_get_blobs_local ON CLUSTER '{cluster}' ( - -- Timestamps - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' Codec(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'When the sentry received the event' Codec(DoubleDelta, ZSTD(1)), - requested_date_time DateTime64(3) COMMENT 'Timestamp when the engine_getBlobs call was initiated' Codec(DoubleDelta, ZSTD(1)), - - -- Timing - duration_ms UInt64 COMMENT 'How long the engine_getBlobs call took in milliseconds' Codec(ZSTD(1)), - - -- Beacon context - slot UInt32 COMMENT 'Slot number of the beacon block being reconstructed' Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime COMMENT 'The wall clock time when the slot started' Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 COMMENT 'Epoch number derived from the slot' Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime COMMENT 'The wall clock time when the epoch started' Codec(DoubleDelta, ZSTD(1)), - block_root FixedString(66) COMMENT 'Root of the beacon block (hex encoded with 0x prefix)' Codec(ZSTD(1)), - parent_block_root FixedString(66) COMMENT 'Root of the parent beacon block (hex encoded with 0x prefix)' Codec(ZSTD(1)), - - -- Request details - requested_count UInt32 COMMENT 'Number of versioned hashes requested' Codec(ZSTD(1)), - versioned_hashes Array(FixedString(66)) COMMENT 'List of versioned hashes requested (derived from KZG commitments)' Codec(ZSTD(1)), - - -- Response from EL - returned_count UInt32 COMMENT 'Number of non-null blobs returned' Codec(ZSTD(1)), - status LowCardinality(String) COMMENT 'Result status (SUCCESS, PARTIAL, EMPTY, UNSUPPORTED, ERROR)', - error_message Nullable(String) COMMENT 'Error details if status is ERROR or UNSUPPORTED' Codec(ZSTD(1)), - - -- Meta - method_version LowCardinality(String) COMMENT 'Version of the engine_getBlobs method (e.g., V1, V2)', - - -- Execution client metadata - meta_execution_version LowCardinality(String) COMMENT 'Full execution client version string from web3_clientVersion RPC', - meta_execution_implementation LowCardinality(String) COMMENT 'Execution client implementation name (e.g., Geth, Nethermind, Besu, Reth, Erigon)', - meta_execution_version_major LowCardinality(String) COMMENT 'Execution client major version number', - meta_execution_version_minor LowCardinality(String) COMMENT 'Execution client minor version number', - meta_execution_version_patch LowCardinality(String) COMMENT 'Execution client patch version number', - - -- Standard metadata fields - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' Codec(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' Codec(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name, block_root, event_date_time) COMMENT 'Contains timing and instrumentation data for engine_getBlobs calls between the consensus and execution layer.'; - -CREATE TABLE consensus_engine_api_get_blobs ON CLUSTER '{cluster}' AS consensus_engine_api_get_blobs_local -ENGINE = Distributed( - '{cluster}', - default, - consensus_engine_api_get_blobs_local, - cityHash64( - slot_start_date_time, - meta_network_name, - meta_client_name, - block_root, - event_date_time - ) -); diff --git a/deploy/migrations/clickhouse/087_mev_relay_proposer_payload_delivered_value.down.sql b/deploy/migrations/clickhouse/087_mev_relay_proposer_payload_delivered_value.down.sql deleted file mode 100644 index 65b8b65e..00000000 --- a/deploy/migrations/clickhouse/087_mev_relay_proposer_payload_delivered_value.down.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE default.mev_relay_proposer_payload_delivered_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS `value`; - -ALTER TABLE default.mev_relay_proposer_payload_delivered ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS `value`; diff --git a/deploy/migrations/clickhouse/087_mev_relay_proposer_payload_delivered_value.up.sql b/deploy/migrations/clickhouse/087_mev_relay_proposer_payload_delivered_value.up.sql deleted file mode 100644 index d1789d75..00000000 --- a/deploy/migrations/clickhouse/087_mev_relay_proposer_payload_delivered_value.up.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE default.mev_relay_proposer_payload_delivered_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS `value` UInt256 COMMENT 'The bid value in wei' CODEC(ZSTD(1)) AFTER `gas_used`; - -ALTER TABLE default.mev_relay_proposer_payload_delivered ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS `value` UInt256 COMMENT 'The bid value in wei' CODEC(ZSTD(1)) AFTER `gas_used`; diff --git a/deploy/migrations/clickhouse/088_transactions_structlog_call_to.down.sql b/deploy/migrations/clickhouse/088_transactions_structlog_call_to.down.sql deleted file mode 100644 index 0e4a3cf8..00000000 --- a/deploy/migrations/clickhouse/088_transactions_structlog_call_to.down.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - DROP COLUMN gas_used; - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - DROP COLUMN gas_used; diff --git a/deploy/migrations/clickhouse/088_transactions_structlog_call_to.up.sql b/deploy/migrations/clickhouse/088_transactions_structlog_call_to.up.sql deleted file mode 100644 index b307b7af..00000000 --- a/deploy/migrations/clickhouse/088_transactions_structlog_call_to.up.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - ADD COLUMN gas_used UInt64 DEFAULT 0 COMMENT 'Actual gas consumed (computed from consecutive gas values)' CODEC(ZSTD(1)) AFTER gas_cost; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - ADD COLUMN gas_used UInt64 DEFAULT 0 COMMENT 'Actual gas consumed (computed from consecutive gas values)' CODEC(ZSTD(1)) AFTER gas_cost; diff --git a/deploy/migrations/clickhouse/089_table_comments.down.sql b/deploy/migrations/clickhouse/089_table_comments.down.sql deleted file mode 100644 index 0090bf56..00000000 --- a/deploy/migrations/clickhouse/089_table_comments.down.sql +++ /dev/null @@ -1,629 +0,0 @@ --- Rollback: Restore original comments for all tables --- This reverses the comprehensive table comment improvements - --- ============================================================================ --- Rollback: beacon_api_eth_v1_events_* tables --- ============================================================================ - --- beacon_api_eth_v1_events_head -ALTER TABLE default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - -ALTER TABLE default.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "head" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - --- beacon_api_eth_v1_events_block -ALTER TABLE default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - -ALTER TABLE default.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "block" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - --- beacon_api_eth_v1_events_attestation -ALTER TABLE default.beacon_api_eth_v1_events_attestation ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "attestation" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - -ALTER TABLE default.beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "attestation" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - --- beacon_api_eth_v1_events_voluntary_exit -ALTER TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "voluntary exit" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- beacon_api_eth_v1_events_finalized_checkpoint -ALTER TABLE default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "finalized checkpoint" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- beacon_api_eth_v1_events_chain_reorg -ALTER TABLE default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slots between when the reorg occurred and when the sentry received the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "chain reorg" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'Difference in slots between when the reorg occurred and when the sentry received the event'; - --- beacon_api_eth_v1_events_contribution_and_proof -ALTER TABLE default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN contribution_propagation_slot_start_diff 'Difference in slots between when the contribution occurred and when the sentry received the event'; - -ALTER TABLE default.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "contribution and proof" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN contribution_propagation_slot_start_diff 'Difference in slots between when the contribution occurred and when the sentry received the event'; - --- beacon_api_eth_v1_events_blob_sidecar -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "blob_sidecar" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - --- beacon_api_eth_v1_events_data_column_sidecar -ALTER TABLE default.beacon_api_eth_v1_events_data_column_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "data_column_sidecar" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - -ALTER TABLE default.beacon_api_eth_v1_events_data_column_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "data_column_sidecar" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - --- beacon_api_eth_v1_events_block_gossip -ALTER TABLE default.beacon_api_eth_v1_events_block_gossip ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "block_gossip" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; - -ALTER TABLE default.beacon_api_eth_v1_events_block_gossip_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API eventstream "block_gossip" data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN propagation_slot_start_diff 'The difference between the event_date_time and the slot_start_date_time'; --- Rollback: Restore original comments for beacon_api_eth_v1_* and beacon_api_eth_v2_* tables (non-event-stream) - --- beacon_api_eth_v1_beacon_committee -ALTER TABLE default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/beacon/states/{state_id}/committees data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- beacon_api_eth_v1_validator_attestation_data -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/validator/attestation_data data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN request_duration 'The duration of the request', -COMMENT COLUMN request_slot_start_diff 'The difference between the request time and the slot start time'; - -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v1/validator/attestation_data data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN request_duration 'The duration of the request', -COMMENT COLUMN request_slot_start_diff 'The difference between the request time and the slot start time'; - --- beacon_api_eth_v1_proposer_duty -ALTER TABLE default.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a proposer duty from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a proposer duty from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- beacon_api_eth_v2_beacon_block -ALTER TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v2/beacon/blocks/{block_id} data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- beacon_api_eth_v3_validator_block -ALTER TABLE default.beacon_api_eth_v3_validator_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v3/validator/blocks/{slot} data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.beacon_api_eth_v3_validator_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API /eth/v3/validator/blocks/{slot} data from each sentry client attached to a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; --- Rollback: Restore original comments for canonical_beacon_block* tables - --- canonical_beacon_block -ALTER TABLE default.canonical_beacon_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon block from a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon block from a beacon node.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_block_attester_slashing -ALTER TABLE default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains attester slashing from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains attester slashing from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_block_deposit -ALTER TABLE default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a deposit from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a deposit from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_block_execution_transaction -ALTER TABLE default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution transaction from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution transaction from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_block_voluntary_exit -ALTER TABLE default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a voluntary exit from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a voluntary exit from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_block_withdrawal -ALTER TABLE default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a withdrawal from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a withdrawal from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; --- Rollback: Restore original comments for canonical_beacon_* remaining tables - --- canonical_beacon_validators -ALTER TABLE default.canonical_beacon_validators ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a validator state for an epoch.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_validators_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a validator state for an epoch.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_committee -ALTER TABLE default.canonical_beacon_committee ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains canonical beacon API /eth/v1/beacon/states/{state_id}/committees data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_committee_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains canonical beacon API /eth/v1/beacon/states/{state_id}/committees data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_proposer_duty -ALTER TABLE default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a proposer duty from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a proposer duty from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_elaborated_attestation -ALTER TABLE default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains elaborated attestations from beacon blocks.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains elaborated attestations from beacon blocks.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- canonical_beacon_blob_sidecar -ALTER TABLE default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a blob sidecar from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a blob sidecar from a beacon block.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- Rollback: Restore original comments for libp2p_gossipsub_* tables - --- libp2p_gossipsub_beacon_block -ALTER TABLE default.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub beacon block data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub beacon block data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_gossipsub_beacon_attestation -ALTER TABLE default.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub beacon attestation data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub beacon attestation data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_gossipsub_blob_sidecar -ALTER TABLE default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub blob sidecar data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub blob sidecar data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_gossipsub_data_column_sidecar -ALTER TABLE default.libp2p_gossipsub_data_column_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub data column sidecar data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_gossipsub_data_column_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub data column sidecar data', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_gossipsub_aggregate_and_proof -ALTER TABLE default.libp2p_gossipsub_aggregate_and_proof ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub aggregate and proof data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_gossipsub_aggregate_and_proof_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Table for libp2p gossipsub aggregate and proof data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- Rollback: Restore original comments for libp2p peer/connection tables - --- libp2p_peer -ALTER TABLE default.libp2p_peer ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the original peer id of a seahashed peer_id + meta_network_name, commonly seen in other tables as the field peer_id_unique_key'; - -ALTER TABLE default.libp2p_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the original peer id of a seahashed peer_id + meta_network_name, commonly seen in other tables as the field peer_id_unique_key'; - --- libp2p_add_peer -ALTER TABLE default.libp2p_add_peer ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the peers added to the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_add_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the peers added to the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_remove_peer -ALTER TABLE default.libp2p_remove_peer ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the peers removed from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_remove_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the peers removed from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_connected -ALTER TABLE default.libp2p_connected ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the CONNECTED events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_connected_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the CONNECTED events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_disconnected -ALTER TABLE default.libp2p_disconnected ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the DISCONNECTED events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_disconnected_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the DISCONNECTED events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_handle_status -ALTER TABLE default.libp2p_handle_status ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the status handling events for libp2p peers.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_handle_status_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the status handling events for libp2p peers.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_handle_metadata -ALTER TABLE default.libp2p_handle_metadata ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the metadata handling events for libp2p peers.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_handle_metadata_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the metadata handling events for libp2p peers.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_synthetic_heartbeat -ALTER TABLE default.libp2p_synthetic_heartbeat ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains heartbeat events from libp2p peers', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_synthetic_heartbeat_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains heartbeat events from libp2p peers', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- Rollback: Restore original comments for libp2p RPC tables - --- libp2p_recv_rpc -ALTER TABLE default.libp2p_recv_rpc ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC messages received by the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_recv_rpc_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC messages received by the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_send_rpc -ALTER TABLE default.libp2p_send_rpc ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC messages sent by the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_send_rpc_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC messages sent by the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_drop_rpc -ALTER TABLE default.libp2p_drop_rpc ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC messages dropped by the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_drop_rpc_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC messages dropped by the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_rpc_meta_message -ALTER TABLE default.libp2p_rpc_meta_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC meta messages from the peer', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_rpc_meta_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC meta messages from the peer', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_rpc_meta_subscription -ALTER TABLE default.libp2p_rpc_meta_subscription ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC subscriptions from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_rpc_meta_subscription_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the RPC subscriptions from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_rpc_meta_control_ihave -ALTER TABLE default.libp2p_rpc_meta_control_ihave ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "I have" control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_rpc_meta_control_ihave_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "I have" control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_rpc_meta_control_iwant -ALTER TABLE default.libp2p_rpc_meta_control_iwant ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "I want" control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_rpc_meta_control_iwant_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "I want" control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_rpc_meta_control_graft -ALTER TABLE default.libp2p_rpc_meta_control_graft ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "Graft" control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_rpc_meta_control_graft_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "Graft" control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_rpc_meta_control_prune -ALTER TABLE default.libp2p_rpc_meta_control_prune ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "Prune" control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_rpc_meta_control_prune_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the "Prune" control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_rpc_meta_control_idontwant -ALTER TABLE default.libp2p_rpc_meta_control_idontwant ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the IDONTWANT control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_rpc_meta_control_idontwant_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the IDONTWANT control messages from the peer.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- Rollback: Restore original comments for libp2p topic/message event tables - --- libp2p_join -ALTER TABLE default.libp2p_join ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the JOIN events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_join_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the JOIN events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_leave -ALTER TABLE default.libp2p_leave ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the LEAVE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_leave_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the LEAVE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_graft -ALTER TABLE default.libp2p_graft ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the GRAFT events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_graft_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the GRAFT events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_prune -ALTER TABLE default.libp2p_prune ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the PRUNE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_prune_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the PRUNE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_deliver_message -ALTER TABLE default.libp2p_deliver_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the DELIVER_MESSAGE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_deliver_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the DELIVER_MESSAGE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_publish_message -ALTER TABLE default.libp2p_publish_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the PUBLISH_MESSAGE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_publish_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the PUBLISH_MESSAGE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_duplicate_message -ALTER TABLE default.libp2p_duplicate_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the DUPLICATE_MESSAGE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_duplicate_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the DUPLICATE_MESSAGE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- libp2p_reject_message -ALTER TABLE default.libp2p_reject_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the REJECT_MESSAGE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.libp2p_reject_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains the details of the REJECT_MESSAGE events from the libp2p client.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- Rollback: Restore original comments for mempool_*, mev_relay_*, node_record_* tables - --- mempool_transaction (no original COMMENT) --- Leaving unchanged as there was no original table comment - --- mempool_dumpster_transaction -ALTER TABLE default.mempool_dumpster_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions from mempool dumpster dataset. Following the parquet schema with some additions'; - -ALTER TABLE default.mempool_dumpster_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions from mempool dumpster dataset. Following the parquet schema with some additions'; - --- block_native_mempool_transaction -ALTER TABLE default.block_native_mempool_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions from block native mempool dataset'; - -ALTER TABLE default.block_native_mempool_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions from block native mempool dataset'; - --- mev_relay_bid_trace -ALTER TABLE default.mev_relay_bid_trace ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains MEV relay block bids data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.mev_relay_bid_trace_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains MEV relay block bids data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- mev_relay_proposer_payload_delivered -ALTER TABLE default.mev_relay_proposer_payload_delivered ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains MEV relay proposer payload delivered data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.mev_relay_proposer_payload_delivered_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains MEV relay proposer payload delivered data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- mev_relay_validator_registration -ALTER TABLE default.mev_relay_validator_registration ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains MEV relay validator registrations data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.mev_relay_validator_registration_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains MEV relay validator registrations data.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- node_record_execution -ALTER TABLE default.node_record_execution ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution node records discovered by the Xatu discovery module.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.node_record_execution_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution node records discovered by the Xatu discovery module.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- node_record_consensus -ALTER TABLE default.node_record_consensus ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains consensus node records discovered by the Xatu discovery module.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - -ALTER TABLE default.node_record_consensus_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains consensus node records discovered by the Xatu discovery module.', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event'; - --- Rollback: Restore original comments for remaining tables - --- blob_submitter -ALTER TABLE default.blob_submitter ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains blob submitter address to name mappings.'; - -ALTER TABLE default.blob_submitter_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains blob submitter address to name mappings.'; - --- ethseer_validator_entity -ALTER TABLE default.ethseer_validator_entity ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a mapping of validators to entities'; - -ALTER TABLE default.ethseer_validator_entity_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains a mapping of validators to entities'; - diff --git a/deploy/migrations/clickhouse/089_table_comments.up.sql b/deploy/migrations/clickhouse/089_table_comments.up.sql deleted file mode 100644 index d7253f52..00000000 --- a/deploy/migrations/clickhouse/089_table_comments.up.sql +++ /dev/null @@ -1,643 +0,0 @@ --- Comprehensive table comment improvements across all Xatu ClickHouse tables --- Improves table and column comments for succinctness, correctness, and partition key awareness --- Covers: beacon_api_*, canonical_beacon_*, libp2p_*, mempool_*, mev_relay_*, node_record_*, and reference tables - --- ============================================================================ --- Batch 1: beacon_api_eth_v1_events_* table comments --- ============================================================================ - --- beacon_api_eth_v1_events_head -ALTER TABLE default.beacon_api_eth_v1_events_head ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures head events. Each row represents a `head` event from the Beacon API `/eth/v1/events?topics=head`, indicating the chain''s canonical head has been updated. Sentry adds client metadata and propagation timing. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - -ALTER TABLE default.beacon_api_eth_v1_events_head_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures head events. Each row represents a `head` event from the Beacon API `/eth/v1/events?topics=head`, indicating the chain''s canonical head has been updated. Sentry adds client metadata and propagation timing. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - --- beacon_api_eth_v1_events_block -ALTER TABLE default.beacon_api_eth_v1_events_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures block events. Each row represents a `block` event from the Beacon API `/eth/v1/events?topics=block`, indicating a new block has been imported. Sentry adds client metadata and propagation timing. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - -ALTER TABLE default.beacon_api_eth_v1_events_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures block events. Each row represents a `block` event from the Beacon API `/eth/v1/events?topics=block`, indicating a new block has been imported. Sentry adds client metadata and propagation timing. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - --- beacon_api_eth_v1_events_attestation -ALTER TABLE default.beacon_api_eth_v1_events_attestation ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures attestation events. Each row represents an `attestation` event from the Beacon API `/eth/v1/events?topics=attestation`. High-volume table - filter by `slot_start_date_time` and `meta_network_name` to avoid full scans. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - -ALTER TABLE default.beacon_api_eth_v1_events_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures attestation events. Each row represents an `attestation` event from the Beacon API `/eth/v1/events?topics=attestation`. High-volume table - filter by `slot_start_date_time` and `meta_network_name` to avoid full scans. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - --- beacon_api_eth_v1_events_voluntary_exit -ALTER TABLE default.beacon_api_eth_v1_events_voluntary_exit ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures voluntary exit events. Each row represents a `voluntary_exit` event from the Beacon API `/eth/v1/events?topics=voluntary_exit`, when a validator initiates an exit. Partition: monthly by `wallclock_epoch_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients'; - -ALTER TABLE default.beacon_api_eth_v1_events_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures voluntary exit events. Each row represents a `voluntary_exit` event from the Beacon API `/eth/v1/events?topics=voluntary_exit`, when a validator initiates an exit. Partition: monthly by `wallclock_epoch_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients'; - --- beacon_api_eth_v1_events_finalized_checkpoint -ALTER TABLE default.beacon_api_eth_v1_events_finalized_checkpoint ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures finalized checkpoint events. Each row represents a `finalized_checkpoint` event from the Beacon API `/eth/v1/events?topics=finalized_checkpoint`, when the chain finalizes a new epoch. Partition: monthly by `epoch_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients'; - -ALTER TABLE default.beacon_api_eth_v1_events_finalized_checkpoint_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures finalized checkpoint events. Each row represents a `finalized_checkpoint` event from the Beacon API `/eth/v1/events?topics=finalized_checkpoint`, when the chain finalizes a new epoch. Partition: monthly by `epoch_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients'; - --- beacon_api_eth_v1_events_chain_reorg -ALTER TABLE default.beacon_api_eth_v1_events_chain_reorg ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures chain reorg events. Each row represents a `chain_reorg` event from the Beacon API `/eth/v1/events?topics=chain_reorg`, when the beacon node detects a chain reorganization. Includes depth and old/new head info. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - -ALTER TABLE default.beacon_api_eth_v1_events_chain_reorg_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures chain reorg events. Each row represents a `chain_reorg` event from the Beacon API `/eth/v1/events?topics=chain_reorg`, when the beacon node detects a chain reorganization. Includes depth and old/new head info. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - --- beacon_api_eth_v1_events_contribution_and_proof -ALTER TABLE default.beacon_api_eth_v1_events_contribution_and_proof ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures sync committee contribution events. Each row represents a `contribution_and_proof` event from the Beacon API `/eth/v1/events?topics=contribution_and_proof`. Partition: monthly by `contribution_slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN contribution_propagation_slot_start_diff 'Time in milliseconds since the start of the contribution slot when the Sentry received this event'; - -ALTER TABLE default.beacon_api_eth_v1_events_contribution_and_proof_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures sync committee contribution events. Each row represents a `contribution_and_proof` event from the Beacon API `/eth/v1/events?topics=contribution_and_proof`. Partition: monthly by `contribution_slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN contribution_propagation_slot_start_diff 'Time in milliseconds since the start of the contribution slot when the Sentry received this event'; - --- beacon_api_eth_v1_events_blob_sidecar -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures blob sidecar events. Each row represents a `blob_sidecar` event from the Beacon API `/eth/v1/events?topics=blob_sidecar` (EIP-4844) with KZG commitment data. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - -ALTER TABLE default.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures blob sidecar events. Each row represents a `blob_sidecar` event from the Beacon API `/eth/v1/events?topics=blob_sidecar` (EIP-4844) with KZG commitment data. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - --- beacon_api_eth_v1_events_data_column_sidecar -ALTER TABLE default.beacon_api_eth_v1_events_data_column_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures data column sidecar events. Each row represents a `data_column_sidecar` event from the Beacon API `/eth/v1/events?topics=data_column_sidecar` (PeerDAS) with data availability sampling info. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - -ALTER TABLE default.beacon_api_eth_v1_events_data_column_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures data column sidecar events. Each row represents a `data_column_sidecar` event from the Beacon API `/eth/v1/events?topics=data_column_sidecar` (PeerDAS) with data availability sampling info. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - --- beacon_api_eth_v1_events_block_gossip -ALTER TABLE default.beacon_api_eth_v1_events_block_gossip ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures block gossip events. Each row represents a `block_gossip` event from the Beacon API `/eth/v1/events?topics=block_gossip` used for measuring block propagation timing across the network. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; - -ALTER TABLE default.beacon_api_eth_v1_events_block_gossip_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry subscribes to a beacon node''s Beacon API event-stream and captures block gossip events. Each row represents a `block_gossip` event from the Beacon API `/eth/v1/events?topics=block_gossip` used for measuring block propagation timing across the network. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the event. The table contains data from multiple Sentry clients', -COMMENT COLUMN propagation_slot_start_diff 'Time in milliseconds since the start of the slot when the Sentry received this event'; --- Batch 2: beacon_api_eth_v1_* and beacon_api_eth_v2_* table comments (non-event-stream tables) --- Improves table and column comments for succinctness, correctness, and partition key awareness - --- beacon_api_eth_v1_beacon_committee -ALTER TABLE default.beacon_api_eth_v1_beacon_committee ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry calls the Beacon API `/eth/v1/beacon/states/{state_id}/committees` endpoint to fetch committee assignments. Each row contains validator committee assignments for a slot. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients'; - -ALTER TABLE default.beacon_api_eth_v1_beacon_committee_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry calls the Beacon API `/eth/v1/beacon/states/{state_id}/committees` endpoint to fetch committee assignments. Each row contains validator committee assignments for a slot. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients'; - --- beacon_api_eth_v1_validator_attestation_data -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry calls the Beacon API `/eth/v1/validator/attestation_data` endpoint to fetch attestation data. Each row contains attestation data with request timing metrics. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients', -COMMENT COLUMN request_duration 'Time in milliseconds for the Beacon API request to complete', -COMMENT COLUMN request_slot_start_diff 'Time in milliseconds since the start of the slot when the request was made'; - -ALTER TABLE default.beacon_api_eth_v1_validator_attestation_data_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry calls the Beacon API `/eth/v1/validator/attestation_data` endpoint to fetch attestation data. Each row contains attestation data with request timing metrics. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients', -COMMENT COLUMN request_duration 'Time in milliseconds for the Beacon API request to complete', -COMMENT COLUMN request_slot_start_diff 'Time in milliseconds since the start of the slot when the request was made'; - --- beacon_api_eth_v1_proposer_duty -ALTER TABLE default.beacon_api_eth_v1_proposer_duty ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry fetches proposer duties from the Beacon API `/eth/v1/validator/duties/proposer/{epoch}` endpoint. Each row contains which validator is scheduled to propose a block for a given slot. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients'; - -ALTER TABLE default.beacon_api_eth_v1_proposer_duty_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry fetches proposer duties from the Beacon API `/eth/v1/validator/duties/proposer/{epoch}` endpoint. Each row contains which validator is scheduled to propose a block for a given slot. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients'; - --- beacon_api_eth_v2_beacon_block -ALTER TABLE default.beacon_api_eth_v2_beacon_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry calls the Beacon API `/eth/v2/beacon/blocks/{block_id}` endpoint to fetch beacon blocks. Each row contains full beacon block data including execution payload. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients'; - -ALTER TABLE default.beacon_api_eth_v2_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry calls the Beacon API `/eth/v2/beacon/blocks/{block_id}` endpoint to fetch beacon blocks. Each row contains full beacon block data including execution payload. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients'; - --- beacon_api_eth_v3_validator_block -ALTER TABLE default.beacon_api_eth_v3_validator_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry calls the Beacon API `/eth/v3/validator/blocks/{slot}` endpoint. Contains UNSIGNED simulated blocks - what the beacon node would have built if asked to propose at that slot. NOT actual proposed blocks. Useful for MEV research and block building analysis. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients'; - -ALTER TABLE default.beacon_api_eth_v3_validator_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Xatu Sentry calls the Beacon API `/eth/v3/validator/blocks/{slot}` endpoint. Contains UNSIGNED simulated blocks - what the beacon node would have built if asked to propose at that slot. NOT actual proposed blocks. Useful for MEV research and block building analysis. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the Sentry client that collected the data. The table contains data from multiple Sentry clients'; --- Batch 3: canonical_beacon_block* table comments --- Improves table and column comments for succinctness, correctness, and partition key awareness - --- canonical_beacon_block -ALTER TABLE default.canonical_beacon_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains finalized beacon block data. Each row represents a canonical block. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains finalized beacon block data. Each row represents a canonical block. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_block_attester_slashing -ALTER TABLE default.canonical_beacon_block_attester_slashing ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains attester slashings from finalized beacon blocks. Each row represents two conflicting attestations from a slashed validator. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_block_attester_slashing_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains attester slashings from finalized beacon blocks. Each row represents two conflicting attestations from a slashed validator. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_block_deposit -ALTER TABLE default.canonical_beacon_block_deposit ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains validator deposits from finalized beacon blocks. Each row represents a deposit with pubkey, withdrawal credentials, and amount. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_block_deposit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains validator deposits from finalized beacon blocks. Each row represents a deposit with pubkey, withdrawal credentials, and amount. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_block_execution_transaction -ALTER TABLE default.canonical_beacon_block_execution_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution layer transactions from finalized beacon blocks. Each row represents a transaction from the execution payload. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_block_execution_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution layer transactions from finalized beacon blocks. Each row represents a transaction from the execution payload. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_block_voluntary_exit -ALTER TABLE default.canonical_beacon_block_voluntary_exit ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains voluntary exits from finalized beacon blocks. Each row represents a validator initiating an exit. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_block_voluntary_exit_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains voluntary exits from finalized beacon blocks. Each row represents a validator initiating an exit. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_block_withdrawal -ALTER TABLE default.canonical_beacon_block_withdrawal ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains withdrawals from finalized beacon blocks. Each row represents a validator withdrawal with recipient address and amount. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_block_withdrawal_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains withdrawals from finalized beacon blocks. Each row represents a validator withdrawal with recipient address and amount. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; --- Batch 4: canonical_beacon_* remaining table comments --- Improves table and column comments for succinctness, correctness, and partition key awareness - --- canonical_beacon_validators -ALTER TABLE default.canonical_beacon_validators ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains finalized validator state snapshots. Each row represents a validator''s status, balance, and lifecycle epochs at a specific epoch. Partition: monthly by `epoch_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_validators_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains finalized validator state snapshots. Each row represents a validator''s status, balance, and lifecycle epochs at a specific epoch. Partition: monthly by `epoch_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_committee -ALTER TABLE default.canonical_beacon_committee ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains finalized beacon committee assignments. Each row represents a committee with its validator indices for a given slot. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_committee_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains finalized beacon committee assignments. Each row represents a committee with its validator indices for a given slot. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_proposer_duty -ALTER TABLE default.canonical_beacon_proposer_duty ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains finalized proposer duty assignments. Each row represents which validator was scheduled to propose a block for a given slot. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_proposer_duty_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains finalized proposer duty assignments. Each row represents which validator was scheduled to propose a block for a given slot. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_elaborated_attestation -ALTER TABLE default.canonical_beacon_elaborated_attestation ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains elaborated attestations from finalized beacon blocks. Aggregation bits are expanded to actual validator indices. Each row represents an attestation with its participating validators, source/target checkpoints, and position in the block. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_elaborated_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains elaborated attestations from finalized beacon blocks. Aggregation bits are expanded to actual validator indices. Each row represents an attestation with its participating validators, source/target checkpoints, and position in the block. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- canonical_beacon_blob_sidecar -ALTER TABLE default.canonical_beacon_blob_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains blob sidecars from finalized beacon blocks. Each row represents a blob with its KZG commitment, proof, and versioned hash. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.canonical_beacon_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains blob sidecars from finalized beacon blocks. Each row represents a blob with its KZG commitment, proof, and versioned hash. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- Batch 5a: libp2p_gossipsub_* table comments --- Tables populated from deep instrumentation within consensus layer clients (forked Prysm/Lighthouse maintained by ethPandaOps) - --- libp2p_gossipsub_beacon_block -ALTER TABLE default.libp2p_gossipsub_beacon_block ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon block messages received via libp2p gossipsub. Collected from deep instrumentation within forked consensus layer clients (Prysm/Lighthouse). Each row represents a block gossiped on the p2p network with timing and peer metadata. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_gossipsub_beacon_block_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon block messages received via libp2p gossipsub. Collected from deep instrumentation within forked consensus layer clients (Prysm/Lighthouse). Each row represents a block gossiped on the p2p network with timing and peer metadata. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_gossipsub_beacon_attestation -ALTER TABLE default.libp2p_gossipsub_beacon_attestation ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon attestation messages received via libp2p gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents an attestation gossiped on the p2p network with timing and peer metadata. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_gossipsub_beacon_attestation_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon attestation messages received via libp2p gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents an attestation gossiped on the p2p network with timing and peer metadata. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_gossipsub_blob_sidecar -ALTER TABLE default.libp2p_gossipsub_blob_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains blob sidecar messages received via libp2p gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents a blob gossiped on the p2p network with timing and peer metadata. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_gossipsub_blob_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains blob sidecar messages received via libp2p gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents a blob gossiped on the p2p network with timing and peer metadata. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_gossipsub_data_column_sidecar -ALTER TABLE default.libp2p_gossipsub_data_column_sidecar ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains data column sidecar messages received via libp2p gossipsub (PeerDAS). Collected from deep instrumentation within forked consensus layer clients. Each row represents a data column gossiped on the p2p network. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_gossipsub_data_column_sidecar_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains data column sidecar messages received via libp2p gossipsub (PeerDAS). Collected from deep instrumentation within forked consensus layer clients. Each row represents a data column gossiped on the p2p network. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_gossipsub_aggregate_and_proof -ALTER TABLE default.libp2p_gossipsub_aggregate_and_proof ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains aggregate and proof messages received via libp2p gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents an aggregated attestation with its proof. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_gossipsub_aggregate_and_proof_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains aggregate and proof messages received via libp2p gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents an aggregated attestation with its proof. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- Batch 5b: libp2p peer/connection table comments --- Tables populated from deep instrumentation within consensus layer clients (forked Prysm/Lighthouse maintained by ethPandaOps) - --- libp2p_peer -ALTER TABLE default.libp2p_peer ON CLUSTER '{cluster}' -MODIFY COMMENT 'Lookup table mapping seahashed peer_id + network to original peer ID. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.'; - -ALTER TABLE default.libp2p_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Lookup table mapping seahashed peer_id + network to original peer ID. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.'; - --- libp2p_add_peer -ALTER TABLE default.libp2p_add_peer ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains ADD_PEER events when peers are added to the libp2p peer store. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_add_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains ADD_PEER events when peers are added to the libp2p peer store. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_remove_peer -ALTER TABLE default.libp2p_remove_peer ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains REMOVE_PEER events when peers are removed from the libp2p peer store. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_remove_peer_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains REMOVE_PEER events when peers are removed from the libp2p peer store. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_connected -ALTER TABLE default.libp2p_connected ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains CONNECTED events when connections are established to remote peers. Collected from deep instrumentation within forked consensus layer clients. Each row includes remote peer agent info and geolocation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_connected_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains CONNECTED events when connections are established to remote peers. Collected from deep instrumentation within forked consensus layer clients. Each row includes remote peer agent info and geolocation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_disconnected -ALTER TABLE default.libp2p_disconnected ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains DISCONNECTED events when connections to remote peers are closed. Collected from deep instrumentation within forked consensus layer clients. Each row includes remote peer agent info and geolocation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_disconnected_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains DISCONNECTED events when connections to remote peers are closed. Collected from deep instrumentation within forked consensus layer clients. Each row includes remote peer agent info and geolocation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_handle_status -ALTER TABLE default.libp2p_handle_status ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains status protocol handling events (req/resp). Collected from deep instrumentation within forked consensus layer clients. Each row represents a status exchange with a peer including their head and finalized info. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_handle_status_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains status protocol handling events (req/resp). Collected from deep instrumentation within forked consensus layer clients. Each row represents a status exchange with a peer including their head and finalized info. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_handle_metadata -ALTER TABLE default.libp2p_handle_metadata ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains metadata protocol handling events (req/resp). Collected from deep instrumentation within forked consensus layer clients. Each row represents a metadata exchange with a peer including their attnets and syncnets. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_handle_metadata_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains metadata protocol handling events (req/resp). Collected from deep instrumentation within forked consensus layer clients. Each row represents a metadata exchange with a peer including their attnets and syncnets. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_synthetic_heartbeat -ALTER TABLE default.libp2p_synthetic_heartbeat ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains periodic heartbeat snapshots of libp2p peer state. Collected from deep instrumentation within forked consensus layer clients. Each row contains mesh/peer counts and topic subscriptions at that moment. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_synthetic_heartbeat_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains periodic heartbeat snapshots of libp2p peer state. Collected from deep instrumentation within forked consensus layer clients. Each row contains mesh/peer counts and topic subscriptions at that moment. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- Batch 5c: libp2p RPC table comments --- Tables populated from deep instrumentation within consensus layer clients (forked Prysm/Lighthouse maintained by ethPandaOps) - --- libp2p_recv_rpc -ALTER TABLE default.libp2p_recv_rpc ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC messages received from peers. Collected from deep instrumentation within forked consensus layer clients. Control messages are split into separate tables referencing this via rpc_meta_unique_key. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_recv_rpc_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC messages received from peers. Collected from deep instrumentation within forked consensus layer clients. Control messages are split into separate tables referencing this via rpc_meta_unique_key. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_send_rpc -ALTER TABLE default.libp2p_send_rpc ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC messages sent to peers. Collected from deep instrumentation within forked consensus layer clients. Control messages are split into separate tables referencing this via rpc_meta_unique_key. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_send_rpc_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC messages sent to peers. Collected from deep instrumentation within forked consensus layer clients. Control messages are split into separate tables referencing this via rpc_meta_unique_key. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_drop_rpc -ALTER TABLE default.libp2p_drop_rpc ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC messages dropped (not processed) by the peer. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_drop_rpc_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC messages dropped (not processed) by the peer. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_rpc_meta_message -ALTER TABLE default.libp2p_rpc_meta_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC message metadata from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents a message within an RPC with topic and message ID. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_rpc_meta_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC message metadata from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents a message within an RPC with topic and message ID. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_rpc_meta_subscription -ALTER TABLE default.libp2p_rpc_meta_subscription ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC subscription changes from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents a subscribe/unsubscribe action for a topic. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_rpc_meta_subscription_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains RPC subscription changes from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Each row represents a subscribe/unsubscribe action for a topic. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_rpc_meta_control_ihave -ALTER TABLE default.libp2p_rpc_meta_control_ihave ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains IHAVE control messages from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Peers advertise message IDs they have available. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_rpc_meta_control_ihave_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains IHAVE control messages from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Peers advertise message IDs they have available. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_rpc_meta_control_iwant -ALTER TABLE default.libp2p_rpc_meta_control_iwant ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains IWANT control messages from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Peers request specific message IDs. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_rpc_meta_control_iwant_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains IWANT control messages from gossipsub. Collected from deep instrumentation within forked consensus layer clients. Peers request specific message IDs. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_rpc_meta_control_graft -ALTER TABLE default.libp2p_rpc_meta_control_graft ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains GRAFT control messages from gossipsub RPC. Collected from deep instrumentation within forked consensus layer clients. Peers request to join the mesh for a topic. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_rpc_meta_control_graft_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains GRAFT control messages from gossipsub RPC. Collected from deep instrumentation within forked consensus layer clients. Peers request to join the mesh for a topic. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_rpc_meta_control_prune -ALTER TABLE default.libp2p_rpc_meta_control_prune ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains PRUNE control messages from gossipsub RPC. Collected from deep instrumentation within forked consensus layer clients. Peers are removed from the mesh for a topic. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_rpc_meta_control_prune_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains PRUNE control messages from gossipsub RPC. Collected from deep instrumentation within forked consensus layer clients. Peers are removed from the mesh for a topic. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_rpc_meta_control_idontwant -ALTER TABLE default.libp2p_rpc_meta_control_idontwant ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains IDONTWANT control messages from gossipsub RPC. Collected from deep instrumentation within forked consensus layer clients. Peers indicate they do not want certain messages. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_rpc_meta_control_idontwant_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains IDONTWANT control messages from gossipsub RPC. Collected from deep instrumentation within forked consensus layer clients. Peers indicate they do not want certain messages. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- Batch 5d: libp2p topic/message event table comments --- Tables populated from deep instrumentation within consensus layer clients (forked Prysm/Lighthouse maintained by ethPandaOps) - --- libp2p_join -ALTER TABLE default.libp2p_join ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains JOIN events when the local node joins a gossipsub topic. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_join_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains JOIN events when the local node joins a gossipsub topic. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_leave -ALTER TABLE default.libp2p_leave ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains LEAVE events when the local node leaves a gossipsub topic. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_leave_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains LEAVE events when the local node leaves a gossipsub topic. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_graft -ALTER TABLE default.libp2p_graft ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains GRAFT events when a peer joins the mesh for a topic. Collected from deep instrumentation within forked consensus layer clients. Tracks mesh membership changes. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_graft_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains GRAFT events when a peer joins the mesh for a topic. Collected from deep instrumentation within forked consensus layer clients. Tracks mesh membership changes. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_prune -ALTER TABLE default.libp2p_prune ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains PRUNE events when a peer is removed from the mesh for a topic. Collected from deep instrumentation within forked consensus layer clients. Tracks mesh membership changes. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_prune_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains PRUNE events when a peer is removed from the mesh for a topic. Collected from deep instrumentation within forked consensus layer clients. Tracks mesh membership changes. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_deliver_message -ALTER TABLE default.libp2p_deliver_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains DELIVER_MESSAGE events when messages are delivered to local subscribers. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_deliver_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains DELIVER_MESSAGE events when messages are delivered to local subscribers. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_publish_message -ALTER TABLE default.libp2p_publish_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains PUBLISH_MESSAGE events when the local node publishes messages to gossipsub. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_publish_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains PUBLISH_MESSAGE events when the local node publishes messages to gossipsub. Collected from deep instrumentation within forked consensus layer clients. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_duplicate_message -ALTER TABLE default.libp2p_duplicate_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains DUPLICATE_MESSAGE events when a message is received that was already seen. Collected from deep instrumentation within forked consensus layer clients. Useful for analyzing message propagation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_duplicate_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains DUPLICATE_MESSAGE events when a message is received that was already seen. Collected from deep instrumentation within forked consensus layer clients. Useful for analyzing message propagation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- libp2p_reject_message -ALTER TABLE default.libp2p_reject_message ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains REJECT_MESSAGE events when messages fail validation and are rejected. Collected from deep instrumentation within forked consensus layer clients. Includes rejection reason. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.libp2p_reject_message_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains REJECT_MESSAGE events when messages fail validation and are rejected. Collected from deep instrumentation within forked consensus layer clients. Includes rejection reason. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- Batch 6: mempool_*, mev_relay_*, node_record_* table comments - --- mempool_transaction -ALTER TABLE default.mempool_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains pending transactions observed in the mempool. Each row represents a transaction first seen at a specific time with its gas parameters. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.mempool_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains pending transactions observed in the mempool. Each row represents a transaction first seen at a specific time with its gas parameters. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- mempool_dumpster_transaction -ALTER TABLE default.mempool_dumpster_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions imported from the external Mempool Dumpster dataset. Historical mempool data following the parquet schema with additions. Partition: monthly by `event_date_time`.'; - -ALTER TABLE default.mempool_dumpster_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions imported from the external Mempool Dumpster dataset. Historical mempool data following the parquet schema with additions. Partition: monthly by `event_date_time`.'; - --- block_native_mempool_transaction -ALTER TABLE default.block_native_mempool_transaction ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions imported from the external Blocknative mempool dataset. Partition: monthly by `event_date_time`.'; - -ALTER TABLE default.block_native_mempool_transaction_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains transactions imported from the external Blocknative mempool dataset. Partition: monthly by `event_date_time`.'; - --- mev_relay_bid_trace -ALTER TABLE default.mev_relay_bid_trace ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains block bids collected by polling MEV relay data APIs. Each row represents a bid from a builder to a relay with value and block details. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.mev_relay_bid_trace_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains block bids collected by polling MEV relay data APIs. Each row represents a bid from a builder to a relay with value and block details. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- mev_relay_proposer_payload_delivered -ALTER TABLE default.mev_relay_proposer_payload_delivered ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains delivered payloads collected by polling MEV relay data APIs. Each row represents a payload that was delivered to a proposer. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.mev_relay_proposer_payload_delivered_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains delivered payloads collected by polling MEV relay data APIs. Each row represents a payload that was delivered to a proposer. Partition: monthly by `slot_start_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- mev_relay_validator_registration -ALTER TABLE default.mev_relay_validator_registration ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains validator registrations collected by polling MEV relay data APIs. Each row represents a validator registering their fee recipient and gas limit preferences. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.mev_relay_validator_registration_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains validator registrations collected by polling MEV relay data APIs. Each row represents a validator registering their fee recipient and gas limit preferences. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- node_record_execution -ALTER TABLE default.node_record_execution ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution layer node records discovered via discv5 network crawling. Each row represents a discovered node with its ENR, client info, and geolocation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.node_record_execution_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains execution layer node records discovered via discv5 network crawling. Each row represents a discovered node with its ENR, client info, and geolocation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- node_record_consensus -ALTER TABLE default.node_record_consensus ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains consensus layer node records discovered via discv5 network crawling. Each row represents a discovered node with its ENR, client info, and geolocation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - -ALTER TABLE default.node_record_consensus_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains consensus layer node records discovered via discv5 network crawling. Each row represents a discovered node with its ENR, client info, and geolocation. Partition: monthly by `event_date_time`.', -COMMENT COLUMN meta_client_name 'Name of the client that collected the data. The table contains data from multiple clients'; - --- Batch 7: Remaining table comments - --- blob_submitter -ALTER TABLE default.blob_submitter ON CLUSTER '{cluster}' -MODIFY COMMENT 'Lookup table mapping blob submitter addresses to names. Typically used to identify L2 sequencers and rollups submitting blobs to Ethereum. Partition: none (reference table).'; - -ALTER TABLE default.blob_submitter_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Lookup table mapping blob submitter addresses to names. Typically used to identify L2 sequencers and rollups submitting blobs to Ethereum. Partition: none (reference table).'; - --- ethseer_validator_entity -ALTER TABLE default.ethseer_validator_entity ON CLUSTER '{cluster}' -MODIFY COMMENT 'Lookup table mapping validators to entities, imported from Ethseer. Used to label validators by their staking provider or operator. Partition: none (reference table).'; - -ALTER TABLE default.ethseer_validator_entity_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Lookup table mapping validators to entities, imported from Ethseer. Used to label validators by their staking provider or operator. Partition: none (reference table).'; - diff --git a/deploy/migrations/clickhouse/090_execution_engine_events.down.sql b/deploy/migrations/clickhouse/090_execution_engine_events.down.sql deleted file mode 100644 index 6d317e58..00000000 --- a/deploy/migrations/clickhouse/090_execution_engine_events.down.sql +++ /dev/null @@ -1,4 +0,0 @@ -DROP TABLE IF EXISTS execution_engine_new_payload ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS execution_engine_new_payload_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS execution_engine_get_blobs ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS execution_engine_get_blobs_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/090_execution_engine_events.up.sql b/deploy/migrations/clickhouse/090_execution_engine_events.up.sql deleted file mode 100644 index 1387601c..00000000 --- a/deploy/migrations/clickhouse/090_execution_engine_events.up.sql +++ /dev/null @@ -1,134 +0,0 @@ -CREATE TABLE execution_engine_new_payload_local ON CLUSTER '{cluster}' ( - -- Timestamps - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' Codec(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'When the event was received' Codec(DoubleDelta, ZSTD(1)), - requested_date_time DateTime64(3) COMMENT 'Timestamp when the engine_newPayload call was received' Codec(DoubleDelta, ZSTD(1)), - - -- Timing - duration_ms UInt32 COMMENT 'How long the engine_newPayload call took in milliseconds' Codec(ZSTD(1)), - - -- Source - source LowCardinality(String) COMMENT 'Source of the event (SNOOPER, EXECUTION_CLIENT)', - - -- Execution payload details - block_number UInt64 COMMENT 'Execution block number' Codec(DoubleDelta, ZSTD(1)), - block_hash FixedString(66) COMMENT 'Execution block hash (hex encoded with 0x prefix)' Codec(ZSTD(1)), - parent_hash FixedString(66) COMMENT 'Parent execution block hash (hex encoded with 0x prefix)' Codec(ZSTD(1)), - gas_used UInt64 COMMENT 'Total gas used by all transactions in the block' Codec(ZSTD(1)), - gas_limit UInt64 COMMENT 'Gas limit of the block' Codec(ZSTD(1)), - tx_count UInt32 COMMENT 'Number of transactions in the block' Codec(ZSTD(1)), - blob_count UInt32 COMMENT 'Number of blobs in the block' Codec(ZSTD(1)), - - -- Response from EL - status LowCardinality(String) COMMENT 'Payload status returned (VALID, INVALID, SYNCING, ACCEPTED, INVALID_BLOCK_HASH)', - latest_valid_hash Nullable(FixedString(66)) COMMENT 'Latest valid hash when status is INVALID (hex encoded with 0x prefix)' Codec(ZSTD(1)), - validation_error Nullable(String) COMMENT 'Error message when validation fails' Codec(ZSTD(1)), - - -- Meta - method_version LowCardinality(String) COMMENT 'Version of the engine_newPayload method (e.g., V3, V4)', - - -- Execution client metadata - meta_execution_implementation LowCardinality(String) COMMENT 'Implementation of the execution client (e.g., go-ethereum, reth, nethermind)', - meta_execution_version LowCardinality(String) COMMENT 'Version of the execution client', - meta_execution_version_major LowCardinality(String) COMMENT 'Major version number of the execution client', - meta_execution_version_minor LowCardinality(String) COMMENT 'Minor version number of the execution client', - meta_execution_version_patch LowCardinality(String) COMMENT 'Patch version number of the execution client', - - -- Standard metadata fields - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' Codec(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' Codec(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY intDiv(block_number, 5000000) -ORDER BY (block_number, meta_network_name, meta_client_name, block_hash, event_date_time) COMMENT 'Contains timing and instrumentation data for engine_newPayload calls from the execution layer perspective.'; - -CREATE TABLE execution_engine_new_payload ON CLUSTER '{cluster}' AS execution_engine_new_payload_local -ENGINE = Distributed( - '{cluster}', - default, - execution_engine_new_payload_local, - cityHash64( - block_number, - meta_network_name, - meta_client_name, - block_hash, - event_date_time - ) -); - -CREATE TABLE execution_engine_get_blobs_local ON CLUSTER '{cluster}' ( - -- Timestamps - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' Codec(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'When the event was received' Codec(DoubleDelta, ZSTD(1)), - requested_date_time DateTime64(3) COMMENT 'Timestamp when the engine_getBlobs call was received' Codec(DoubleDelta, ZSTD(1)), - - -- Timing - duration_ms UInt32 COMMENT 'How long the engine_getBlobs call took in milliseconds' Codec(ZSTD(1)), - - -- Source - source LowCardinality(String) COMMENT 'Source of the event (SNOOPER, EXECUTION_CLIENT)', - - -- Request details - requested_count UInt32 COMMENT 'Number of versioned hashes requested' Codec(ZSTD(1)), - versioned_hashes Array(FixedString(66)) COMMENT 'List of versioned hashes requested (hex encoded)' Codec(ZSTD(1)), - - -- Response from EL - returned_count UInt32 COMMENT 'Number of non-null blobs returned' Codec(ZSTD(1)), - status LowCardinality(String) COMMENT 'Result status (SUCCESS, PARTIAL, EMPTY, UNSUPPORTED, ERROR)', - error_message Nullable(String) COMMENT 'Error details if status is ERROR or UNSUPPORTED' Codec(ZSTD(1)), - - -- Meta - method_version LowCardinality(String) COMMENT 'Version of the engine_getBlobs method (e.g., V1, V2)', - - -- Execution client metadata - meta_execution_implementation LowCardinality(String) COMMENT 'Implementation of the execution client (e.g., go-ethereum, reth, nethermind)', - meta_execution_version LowCardinality(String) COMMENT 'Version of the execution client', - meta_execution_version_major LowCardinality(String) COMMENT 'Major version number of the execution client', - meta_execution_version_minor LowCardinality(String) COMMENT 'Minor version number of the execution client', - meta_execution_version_patch LowCardinality(String) COMMENT 'Patch version number of the execution client', - - -- Standard metadata fields - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' Codec(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' Codec(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) -PARTITION BY toStartOfMonth(event_date_time) -ORDER BY (event_date_time, meta_network_name, meta_client_name) COMMENT 'Contains timing and instrumentation data for engine_getBlobs calls from the execution layer perspective.'; - -CREATE TABLE execution_engine_get_blobs ON CLUSTER '{cluster}' AS execution_engine_get_blobs_local -ENGINE = Distributed( - '{cluster}', - default, - execution_engine_get_blobs_local, - cityHash64( - event_date_time, - meta_network_name, - meta_client_name - ) -); diff --git a/deploy/migrations/clickhouse/091_canonical_execution_block_gas_limit.down.sql b/deploy/migrations/clickhouse/091_canonical_execution_block_gas_limit.down.sql deleted file mode 100644 index dc578e96..00000000 --- a/deploy/migrations/clickhouse/091_canonical_execution_block_gas_limit.down.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE default.canonical_execution_block ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS `gas_limit`; - -ALTER TABLE default.canonical_execution_block_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS `gas_limit`; diff --git a/deploy/migrations/clickhouse/091_canonical_execution_block_gas_limit.up.sql b/deploy/migrations/clickhouse/091_canonical_execution_block_gas_limit.up.sql deleted file mode 100644 index a01e0146..00000000 --- a/deploy/migrations/clickhouse/091_canonical_execution_block_gas_limit.up.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE default.canonical_execution_block_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS `gas_limit` UInt64 COMMENT 'The block gas limit' CODEC(DoubleDelta, ZSTD(1)) AFTER `gas_used`; - -ALTER TABLE default.canonical_execution_block ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS `gas_limit` UInt64 COMMENT 'The block gas limit' CODEC(DoubleDelta, ZSTD(1)) AFTER `gas_used`; diff --git a/deploy/migrations/clickhouse/092_engine_get_blobs_returned_indexes.down.sql b/deploy/migrations/clickhouse/092_engine_get_blobs_returned_indexes.down.sql deleted file mode 100644 index 78cdfa60..00000000 --- a/deploy/migrations/clickhouse/092_engine_get_blobs_returned_indexes.down.sql +++ /dev/null @@ -1,5 +0,0 @@ -ALTER TABLE execution_engine_get_blobs ON CLUSTER '{cluster}' -DROP COLUMN IF EXISTS returned_blob_indexes; - -ALTER TABLE execution_engine_get_blobs_local ON CLUSTER '{cluster}' -DROP COLUMN IF EXISTS returned_blob_indexes; diff --git a/deploy/migrations/clickhouse/092_engine_get_blobs_returned_indexes.up.sql b/deploy/migrations/clickhouse/092_engine_get_blobs_returned_indexes.up.sql deleted file mode 100644 index d3dc112c..00000000 --- a/deploy/migrations/clickhouse/092_engine_get_blobs_returned_indexes.up.sql +++ /dev/null @@ -1,9 +0,0 @@ -ALTER TABLE execution_engine_get_blobs_local ON CLUSTER '{cluster}' -ADD COLUMN IF NOT EXISTS returned_blob_indexes Array(UInt8) -COMMENT 'Indexes (0-based) of the requested versioned_hashes that were successfully returned' -AFTER returned_count; - -ALTER TABLE execution_engine_get_blobs ON CLUSTER '{cluster}' -ADD COLUMN IF NOT EXISTS returned_blob_indexes Array(UInt8) -COMMENT 'Indexes (0-based) of the requested versioned_hashes that were successfully returned' -AFTER returned_count; diff --git a/deploy/migrations/clickhouse/093_beacon_blob.down.sql b/deploy/migrations/clickhouse/093_beacon_blob.down.sql deleted file mode 100644 index 47b980e9..00000000 --- a/deploy/migrations/clickhouse/093_beacon_blob.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS beacon_api_eth_v1_beacon_blob ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS beacon_api_eth_v1_beacon_blob_local ON CLUSTER '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/093_beacon_blob.up.sql b/deploy/migrations/clickhouse/093_beacon_blob.up.sql deleted file mode 100644 index 2241920c..00000000 --- a/deploy/migrations/clickhouse/093_beacon_blob.up.sql +++ /dev/null @@ -1,76 +0,0 @@ -CREATE TABLE beacon_api_eth_v1_beacon_blob_local on cluster '{cluster}' ( - event_date_time DateTime64(3) Codec(DoubleDelta, ZSTD(1)), - slot UInt32 Codec(DoubleDelta, ZSTD(1)), - slot_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - epoch UInt32 Codec(DoubleDelta, ZSTD(1)), - epoch_start_date_time DateTime Codec(DoubleDelta, ZSTD(1)), - block_root FixedString(66) Codec(ZSTD(1)), - block_parent_root FixedString(66) Codec(ZSTD(1)), - proposer_index UInt32 Codec(ZSTD(1)), - blob_index UInt64 Codec(ZSTD(1)), - kzg_commitment FixedString(98) Codec(ZSTD(1)), - versioned_hash FixedString(66) Codec(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_client_id String CODEC(ZSTD(1)), - meta_client_version LowCardinality(String), - meta_client_implementation LowCardinality(String), - meta_client_os LowCardinality(String), - meta_client_ip Nullable(IPv6) CODEC(ZSTD(1)), - meta_client_geo_city LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) CODEC(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) CODEC(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) CODEC(ZSTD(1)), - meta_network_id Int32 CODEC(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String), - meta_consensus_version LowCardinality(String), - meta_consensus_version_major LowCardinality(String), - meta_consensus_version_minor LowCardinality(String), - meta_consensus_version_patch LowCardinality(String), - meta_consensus_implementation LowCardinality(String), - meta_labels Map(String, String) CODEC(ZSTD(1)) -) Engine = ReplicatedMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}') -PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY (slot_start_date_time, meta_network_name, meta_client_name, block_root, blob_index); - -ALTER TABLE default.beacon_api_eth_v1_beacon_blob_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Contains beacon API blob metadata derived from block blob_kzg_commitments from each sentry client attached to a beacon node.', -COMMENT COLUMN event_date_time 'When the sentry received the event from a beacon node', -COMMENT COLUMN slot 'Slot number in the beacon block', -COMMENT COLUMN slot_start_date_time 'The wall clock time when the slot started', -COMMENT COLUMN epoch 'The epoch number', -COMMENT COLUMN epoch_start_date_time 'The wall clock time when the epoch started', -COMMENT COLUMN block_root 'The beacon block root hash', -COMMENT COLUMN block_parent_root 'The beacon block parent root hash', -COMMENT COLUMN proposer_index 'The validator index of the block proposer', -COMMENT COLUMN blob_index 'The index of the blob within the block', -COMMENT COLUMN kzg_commitment 'The KZG commitment of the blob', -COMMENT COLUMN versioned_hash 'The versioned hash derived from the KZG commitment (key for joining with execution_engine_get_blobs)', -COMMENT COLUMN meta_client_name 'Name of the client that generated the event', -COMMENT COLUMN meta_client_id 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.', -COMMENT COLUMN meta_client_version 'Version of the client that generated the event', -COMMENT COLUMN meta_client_implementation 'Implementation of the client that generated the event', -COMMENT COLUMN meta_client_os 'Operating system of the client that generated the event', -COMMENT COLUMN meta_client_ip 'IP address of the client that generated the event', -COMMENT COLUMN meta_client_geo_city 'City of the client that generated the event', -COMMENT COLUMN meta_client_geo_country 'Country of the client that generated the event', -COMMENT COLUMN meta_client_geo_country_code 'Country code of the client that generated the event', -COMMENT COLUMN meta_client_geo_continent_code 'Continent code of the client that generated the event', -COMMENT COLUMN meta_client_geo_longitude 'Longitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_latitude 'Latitude of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_number 'Autonomous system number of the client that generated the event', -COMMENT COLUMN meta_client_geo_autonomous_system_organization 'Autonomous system organization of the client that generated the event', -COMMENT COLUMN meta_network_id 'Ethereum network ID', -COMMENT COLUMN meta_network_name 'Ethereum network name', -COMMENT COLUMN meta_consensus_version 'Ethereum consensus client version that generated the event', -COMMENT COLUMN meta_consensus_version_major 'Ethereum consensus client major version that generated the event', -COMMENT COLUMN meta_consensus_version_minor 'Ethereum consensus client minor version that generated the event', -COMMENT COLUMN meta_consensus_version_patch 'Ethereum consensus client patch version that generated the event', -COMMENT COLUMN meta_consensus_implementation 'Ethereum consensus client implementation that generated the event', -COMMENT COLUMN meta_labels 'Labels associated with the event'; - -CREATE TABLE beacon_api_eth_v1_beacon_blob on cluster '{cluster}' AS beacon_api_eth_v1_beacon_blob_local -ENGINE = Distributed('{cluster}', default, beacon_api_eth_v1_beacon_blob_local, rand()); diff --git a/deploy/migrations/clickhouse/094_structlog_call_frames.down.sql b/deploy/migrations/clickhouse/094_structlog_call_frames.down.sql deleted file mode 100644 index d60c4f0e..00000000 --- a/deploy/migrations/clickhouse/094_structlog_call_frames.down.sql +++ /dev/null @@ -1,45 +0,0 @@ --- Drop canonical_execution_transaction_structlog_agg tables -DROP TABLE IF EXISTS default.canonical_execution_transaction_structlog_agg ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.canonical_execution_transaction_structlog_agg_local ON CLUSTER '{cluster}'; - -ALTER TABLE admin.execution_block_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS task_count; - -ALTER TABLE admin.execution_block ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS task_count; - -ALTER TABLE admin.execution_block_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS complete; - -ALTER TABLE admin.execution_block ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS complete; - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - ADD COLUMN meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - ADD COLUMN meta_network_id Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)); - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - ADD COLUMN program_counter UInt32 COMMENT 'The program counter' CODEC(Delta, ZSTD(1)); - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - ADD COLUMN program_counter UInt32 COMMENT 'The program counter' CODEC(Delta, ZSTD(1)); - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - DROP COLUMN call_frame_path; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - DROP COLUMN call_frame_path; - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - DROP COLUMN call_frame_id; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - DROP COLUMN call_frame_id; - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - DROP COLUMN gas_self; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - DROP COLUMN gas_self; diff --git a/deploy/migrations/clickhouse/094_structlog_call_frames.up.sql b/deploy/migrations/clickhouse/094_structlog_call_frames.up.sql deleted file mode 100644 index be990cad..00000000 --- a/deploy/migrations/clickhouse/094_structlog_call_frames.up.sql +++ /dev/null @@ -1,91 +0,0 @@ -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - ADD COLUMN gas_self UInt64 DEFAULT 0 COMMENT 'Gas consumed by this opcode only, excludes child frame gas for CALL/CREATE opcodes. sum(gas_self) = total execution gas without double counting' CODEC(ZSTD(1)) AFTER gas_used; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - ADD COLUMN gas_self UInt64 DEFAULT 0 COMMENT 'Gas consumed by this opcode only, excludes child frame gas for CALL/CREATE opcodes. sum(gas_self) = total execution gas without double counting' CODEC(ZSTD(1)) AFTER gas_used; - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - ADD COLUMN call_frame_id UInt32 DEFAULT 0 COMMENT 'Sequential identifier for the call frame within the transaction' CODEC(DoubleDelta, ZSTD(1)) AFTER call_to_address; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - ADD COLUMN call_frame_id UInt32 DEFAULT 0 COMMENT 'Sequential identifier for the call frame within the transaction' CODEC(DoubleDelta, ZSTD(1)) AFTER call_to_address; - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - ADD COLUMN call_frame_path Array(UInt32) DEFAULT [0] COMMENT 'Path of frame IDs from root to current frame' CODEC(ZSTD(1)) AFTER call_frame_id; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - ADD COLUMN call_frame_path Array(UInt32) DEFAULT [0] COMMENT 'Path of frame IDs from root to current frame' CODEC(ZSTD(1)) AFTER call_frame_id; - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - DROP COLUMN program_counter; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - DROP COLUMN program_counter; - -ALTER TABLE canonical_execution_transaction_structlog_local ON CLUSTER '{cluster}' - DROP COLUMN meta_network_id; - -ALTER TABLE canonical_execution_transaction_structlog ON CLUSTER '{cluster}' - DROP COLUMN meta_network_id; - -ALTER TABLE admin.execution_block_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS complete UInt8; - -ALTER TABLE admin.execution_block ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS complete UInt8; - -ALTER TABLE admin.execution_block_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS task_count UInt32; - -ALTER TABLE admin.execution_block ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS task_count UInt32; - --- Create canonical_execution_transaction_structlog_agg table for aggregated structlog data --- This table stores per-frame summary rows (operation='') and per-opcode aggregated rows --- Produces ~50-200x fewer rows than raw structlog table -CREATE TABLE default.canonical_execution_transaction_structlog_agg_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `block_number` UInt64 COMMENT 'The block number' CODEC(DoubleDelta, ZSTD(1)), - `transaction_hash` FixedString(66) COMMENT 'The transaction hash' CODEC(ZSTD(1)), - `transaction_index` UInt32 COMMENT 'The transaction position in the block' CODEC(DoubleDelta, ZSTD(1)), - `call_frame_id` UInt32 COMMENT 'Sequential frame ID within the transaction (0=root)' CODEC(DoubleDelta, ZSTD(1)), - `parent_call_frame_id` Nullable(UInt32) COMMENT 'Parent frame ID (NULL for root frame)' CODEC(ZSTD(1)), - `call_frame_path` Array(UInt32) COMMENT 'Path of frame IDs from root to current frame' CODEC(ZSTD(1)), - `depth` UInt32 COMMENT 'Call nesting depth (0=root)' CODEC(DoubleDelta, ZSTD(1)), - `target_address` Nullable(String) COMMENT 'Contract address being called' CODEC(ZSTD(1)), - `call_type` LowCardinality(String) COMMENT 'Call type: CALL/DELEGATECALL/STATICCALL/CALLCODE/CREATE/CREATE2 (empty for root)', - `operation` LowCardinality(String) COMMENT 'Opcode name for per-opcode rows, empty string for frame summary rows', - `opcode_count` UInt64 COMMENT 'Number of opcodes (total for summary row, count for per-opcode row)' CODEC(ZSTD(1)), - `error_count` UInt64 COMMENT 'Number of errors' CODEC(ZSTD(1)), - `gas` UInt64 COMMENT 'Gas consumed: SUM(gas_self) for per-opcode, frame self gas for summary' CODEC(ZSTD(1)), - `gas_cumulative` UInt64 COMMENT 'Cumulative gas: SUM(gas_used) for per-opcode, frame total for summary' CODEC(ZSTD(1)), - `min_depth` UInt32 COMMENT 'Minimum depth where opcode appeared (per-opcode rows)' CODEC(DoubleDelta, ZSTD(1)), - `max_depth` UInt32 COMMENT 'Maximum depth where opcode appeared (per-opcode rows)' CODEC(DoubleDelta, ZSTD(1)), - `gas_refund` Nullable(UInt64) COMMENT 'Gas refund (root summary row only)' CODEC(ZSTD(1)), - `intrinsic_gas` Nullable(UInt64) COMMENT 'Intrinsic gas (root summary row only, computed)' CODEC(ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name' -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY intDiv(block_number, 201600) -- roughly 1 month of blocks -ORDER BY - ( - block_number, - meta_network_name, - transaction_hash, - call_frame_id, - operation - ) COMMENT 'Aggregated EVM execution data. Summary rows (operation="") contain frame metadata. Per-opcode rows contain aggregated gas/count per (frame, opcode).'; - -CREATE TABLE default.canonical_execution_transaction_structlog_agg ON CLUSTER '{cluster}' AS default.canonical_execution_transaction_structlog_agg_local ENGINE = Distributed( - '{cluster}', - default, - canonical_execution_transaction_structlog_agg_local, - cityHash64( - block_number, - meta_network_name, - transaction_hash, - call_frame_id - ) -); diff --git a/deploy/migrations/clickhouse/095_canonical_beacon_sync_committee.down.sql b/deploy/migrations/clickhouse/095_canonical_beacon_sync_committee.down.sql deleted file mode 100644 index ed4988b6..00000000 --- a/deploy/migrations/clickhouse/095_canonical_beacon_sync_committee.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.canonical_beacon_sync_committee ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS default.canonical_beacon_sync_committee_local ON cluster '{cluster}' SYNC; diff --git a/deploy/migrations/clickhouse/095_canonical_beacon_sync_committee.up.sql b/deploy/migrations/clickhouse/095_canonical_beacon_sync_committee.up.sql deleted file mode 100644 index a14cb41e..00000000 --- a/deploy/migrations/clickhouse/095_canonical_beacon_sync_committee.up.sql +++ /dev/null @@ -1,51 +0,0 @@ -CREATE TABLE default.canonical_beacon_sync_committee_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number for when the sync committee is active' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `sync_committee_period` UInt64 COMMENT 'The sync committee period number' CODEC(DoubleDelta, ZSTD(1)), - `validator_aggregates` Array(Array(UInt32)) COMMENT 'The validator indices grouped by subcommittee (64 groups of 8)' CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(epoch_start_date_time) -ORDER BY - ( - epoch_start_date_time, - meta_network_name, - sync_committee_period - ) COMMENT 'Contains canonical beacon API /eth/v1/beacon/states/{state_id}/sync_committees data.'; - -CREATE TABLE default.canonical_beacon_sync_committee ON CLUSTER '{cluster}' AS default.canonical_beacon_sync_committee_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_sync_committee_local, - cityHash64( - epoch_start_date_time, - meta_network_name, - sync_committee_period - ) -); diff --git a/deploy/migrations/clickhouse/096_canonical_beacon_block_sync_aggregate.down.sql b/deploy/migrations/clickhouse/096_canonical_beacon_block_sync_aggregate.down.sql deleted file mode 100644 index 1b8232be..00000000 --- a/deploy/migrations/clickhouse/096_canonical_beacon_block_sync_aggregate.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS default.canonical_beacon_block_sync_aggregate ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS default.canonical_beacon_block_sync_aggregate_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/096_canonical_beacon_block_sync_aggregate.up.sql b/deploy/migrations/clickhouse/096_canonical_beacon_block_sync_aggregate.up.sql deleted file mode 100644 index c75e0360..00000000 --- a/deploy/migrations/clickhouse/096_canonical_beacon_block_sync_aggregate.up.sql +++ /dev/null @@ -1,59 +0,0 @@ -CREATE TABLE default.canonical_beacon_block_sync_aggregate_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime COMMENT 'Timestamp when the record was last updated' CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) COMMENT 'When the sentry received the event from a beacon node' CODEC(DoubleDelta, ZSTD(1)), - `slot` UInt32 COMMENT 'Slot number in the beacon chain' CODEC(DoubleDelta, ZSTD(1)), - `slot_start_date_time` DateTime COMMENT 'The wall clock time when the slot started' CODEC(DoubleDelta, ZSTD(1)), - `epoch` UInt32 COMMENT 'The epoch number in the beacon chain' CODEC(DoubleDelta, ZSTD(1)), - `epoch_start_date_time` DateTime COMMENT 'The wall clock time when the epoch started' CODEC(DoubleDelta, ZSTD(1)), - `block_root` FixedString(66) COMMENT 'The root hash of the beacon block' CODEC(ZSTD(1)), - `block_version` LowCardinality(String) COMMENT 'The version of the beacon block', - `sync_committee_period` UInt64 COMMENT 'The sync committee period number (epoch / 256)' CODEC(DoubleDelta, ZSTD(1)), - `sync_committee_bits` String COMMENT 'Raw 512-bit bitvector as hex string' CODEC(ZSTD(1)), - `sync_committee_signature` String COMMENT 'Aggregated signature from participating validators' CODEC(ZSTD(1)), - `validators_participated` Array(UInt32) COMMENT 'Validator indices that participated (voted)' CODEC(ZSTD(1)), - `validators_missed` Array(UInt32) COMMENT 'Validator indices that missed (did not vote)' CODEC(ZSTD(1)), - `participation_count` UInt16 COMMENT 'Number of validators that participated (0-512)' CODEC(DoubleDelta, ZSTD(1)), - `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', - `meta_client_id` String COMMENT 'Unique Session ID of the client that generated the event. This changes every time the client is restarted.' CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', - `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - `meta_client_os` LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - `meta_client_ip` Nullable(IPv6) COMMENT 'IP address of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) COMMENT 'City of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) COMMENT 'Country of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) COMMENT 'Country code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) COMMENT 'Continent code of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) COMMENT 'Longitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) COMMENT 'Latitude of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' CODEC(ZSTD(1)), - `meta_network_id` Int32 COMMENT 'Ethereum network ID' CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) COMMENT 'Ethereum network name', - `meta_consensus_version` LowCardinality(String) COMMENT 'Ethereum consensus client version that generated the event', - `meta_consensus_version_major` LowCardinality(String) COMMENT 'Ethereum consensus client major version that generated the event', - `meta_consensus_version_minor` LowCardinality(String) COMMENT 'Ethereum consensus client minor version that generated the event', - `meta_consensus_version_patch` LowCardinality(String) COMMENT 'Ethereum consensus client patch version that generated the event', - `meta_consensus_implementation` LowCardinality(String) COMMENT 'Ethereum consensus client implementation that generated the event', - `meta_labels` Map(String, String) COMMENT 'Labels associated with the event' CODEC(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/{database}/tables/{table}/{shard}', - '{replica}', - updated_date_time -) PARTITION BY toStartOfMonth(slot_start_date_time) -ORDER BY - ( - slot_start_date_time, - meta_network_name, - slot - ) COMMENT 'Contains canonical beacon block sync aggregate data with expanded validator participation.'; - -CREATE TABLE default.canonical_beacon_block_sync_aggregate ON CLUSTER '{cluster}' AS default.canonical_beacon_block_sync_aggregate_local ENGINE = Distributed( - '{cluster}', - default, - canonical_beacon_block_sync_aggregate_local, - cityHash64( - slot_start_date_time, - meta_network_name, - slot - ) -); diff --git a/deploy/migrations/clickhouse/097_observoor_initialize.down.sql b/deploy/migrations/clickhouse/097_observoor_initialize.down.sql deleted file mode 100644 index 1e384596..00000000 --- a/deploy/migrations/clickhouse/097_observoor_initialize.down.sql +++ /dev/null @@ -1,89 +0,0 @@ -DROP DATABASE IF EXISTS observoor ON CLUSTER '{cluster}'; - --- Observoor ClickHouse Schema Teardown - --- Raw events -DROP TABLE IF EXISTS observoor.raw_events ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.raw_events_local ON CLUSTER '{cluster}'; - --- Sync state -DROP TABLE IF EXISTS observoor.sync_state ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.sync_state_local ON CLUSTER '{cluster}'; - --- Syscall latency tables -DROP TABLE IF EXISTS observoor.syscall_read ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_read_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_write ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_write_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_futex ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_futex_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_mmap ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_mmap_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_epoll_wait ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_epoll_wait_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_fsync ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_fsync_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_fdatasync ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_fdatasync_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_pwrite ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.syscall_pwrite_local ON CLUSTER '{cluster}'; - --- Scheduler latency tables -DROP TABLE IF EXISTS observoor.sched_on_cpu ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.sched_on_cpu_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.sched_off_cpu ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.sched_off_cpu_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.sched_runqueue ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.sched_runqueue_local ON CLUSTER '{cluster}'; - --- Memory latency tables -DROP TABLE IF EXISTS observoor.mem_reclaim ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.mem_reclaim_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.mem_compaction ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.mem_compaction_local ON CLUSTER '{cluster}'; - --- Disk latency table -DROP TABLE IF EXISTS observoor.disk_latency ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.disk_latency_local ON CLUSTER '{cluster}'; - --- Memory counter tables -DROP TABLE IF EXISTS observoor.page_fault_major ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.page_fault_major_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.page_fault_minor ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.page_fault_minor_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.swap_in ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.swap_in_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.swap_out ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.swap_out_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.oom_kill ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.oom_kill_local ON CLUSTER '{cluster}'; - --- Process counter tables -DROP TABLE IF EXISTS observoor.fd_open ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.fd_open_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.fd_close ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.fd_close_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.process_exit ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.process_exit_local ON CLUSTER '{cluster}'; - --- Network counter tables -DROP TABLE IF EXISTS observoor.tcp_state_change ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.tcp_state_change_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.net_io ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.net_io_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.tcp_retransmit ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.tcp_retransmit_local ON CLUSTER '{cluster}'; - --- Disk counter tables -DROP TABLE IF EXISTS observoor.disk_bytes ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.disk_bytes_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.block_merge ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.block_merge_local ON CLUSTER '{cluster}'; - --- Gauge tables -DROP TABLE IF EXISTS observoor.tcp_rtt ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.tcp_rtt_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.tcp_cwnd ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.tcp_cwnd_local ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.disk_queue_depth ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.disk_queue_depth_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/097_observoor_initialize.up.sql b/deploy/migrations/clickhouse/097_observoor_initialize.up.sql deleted file mode 100644 index c3e31071..00000000 --- a/deploy/migrations/clickhouse/097_observoor_initialize.up.sql +++ /dev/null @@ -1,1119 +0,0 @@ -CREATE DATABASE IF NOT EXISTS observoor ON CLUSTER '{cluster}'; - --------------------------------------------------------------------------------- --- RAW EVENTS --- One row per kernel event. Use for debugging and detailed analysis. --- Expected volume: ~20 GB/day at 36k events/sec --------------------------------------------------------------------------------- - -CREATE TABLE observoor.raw_events_local ON CLUSTER '{cluster}' ( - -- Timing - timestamp_ns UInt64 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot UInt64 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - - -- Sync state - cl_syncing Bool CODEC(ZSTD(1)), - el_optimistic Bool CODEC(ZSTD(1)), - el_offline Bool CODEC(ZSTD(1)), - - -- Process identification - pid UInt32 CODEC(ZSTD(1)), - tid UInt32 CODEC(ZSTD(1)), - event_type LowCardinality(String), - client_type LowCardinality(String), - - -- Common fields - latency_ns UInt64 CODEC(ZSTD(1)), - bytes Int64 CODEC(ZSTD(1)), - - -- Network fields - src_port UInt16 CODEC(ZSTD(1)), - dst_port UInt16 CODEC(ZSTD(1)), - - -- File descriptor fields - fd Int32 CODEC(ZSTD(1)), - filename String CODEC(ZSTD(1)), - - -- Scheduler fields - voluntary Bool CODEC(ZSTD(1)), - on_cpu_ns UInt64 CODEC(ZSTD(1)), - runqueue_ns UInt64 CODEC(ZSTD(1)), - off_cpu_ns UInt64 CODEC(ZSTD(1)), - - -- Memory fields - major Bool CODEC(ZSTD(1)), - address UInt64 CODEC(ZSTD(1)), - pages UInt64 CODEC(ZSTD(1)), - - -- Disk I/O fields - rw UInt8 CODEC(ZSTD(1)), - queue_depth UInt32 CODEC(ZSTD(1)), - device_id UInt32 CODEC(ZSTD(1)), - - -- TCP fields - tcp_state UInt8 CODEC(ZSTD(1)), - tcp_old_state UInt8 CODEC(ZSTD(1)), - tcp_srtt_us UInt32 CODEC(ZSTD(1)), - tcp_cwnd UInt32 CODEC(ZSTD(1)), - - -- Process lifecycle fields - exit_code UInt32 CODEC(ZSTD(1)), - target_pid UInt32 CODEC(ZSTD(1)), - - -- Metadata - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}' -) -PARTITION BY toStartOfMonth(wallclock_slot_start_date_time) -ORDER BY (meta_network_name, wallclock_slot_start_date_time, client_type, event_type, pid); - -ALTER TABLE observoor.raw_events_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Raw eBPF events captured from Ethereum client processes, one row per kernel event.', -COMMENT COLUMN timestamp_ns 'Wall clock time of the event in nanoseconds since Unix epoch', -COMMENT COLUMN wallclock_slot 'Ethereum slot number at the time of the event (from wall clock)', -COMMENT COLUMN wallclock_slot_start_date_time 'Wall clock time when the slot started', -COMMENT COLUMN cl_syncing 'Whether the consensus layer was syncing when this event was captured', -COMMENT COLUMN el_optimistic 'Whether the execution layer was in optimistic sync mode when this event was captured', -COMMENT COLUMN el_offline 'Whether the execution layer was unreachable when this event was captured', -COMMENT COLUMN pid 'Process ID of the traced Ethereum client', -COMMENT COLUMN tid 'Thread ID within the traced process', -COMMENT COLUMN event_type 'Type of eBPF event (syscall_read, disk_io, net_tx, etc.)', -COMMENT COLUMN client_type 'Ethereum client implementation (geth, reth, prysm, lighthouse, etc.)', -COMMENT COLUMN latency_ns 'Latency in nanoseconds for syscall and disk I/O events', -COMMENT COLUMN bytes 'Byte count for I/O events', -COMMENT COLUMN src_port 'Source port for network events', -COMMENT COLUMN dst_port 'Destination port for network events', -COMMENT COLUMN fd 'File descriptor number', -COMMENT COLUMN filename 'Filename for fd_open events', -COMMENT COLUMN voluntary 'Whether a context switch was voluntary', -COMMENT COLUMN on_cpu_ns 'Time spent on CPU in nanoseconds before a context switch', -COMMENT COLUMN runqueue_ns 'Time spent waiting in the run queue', -COMMENT COLUMN off_cpu_ns 'Time spent off CPU', -COMMENT COLUMN major 'Whether a page fault was a major fault', -COMMENT COLUMN address 'Faulting address for page fault events', -COMMENT COLUMN pages 'Number of pages for swap events', -COMMENT COLUMN rw 'Read (0) or write (1) for disk I/O', -COMMENT COLUMN queue_depth 'Block device queue depth at time of I/O', -COMMENT COLUMN device_id 'Block device ID (major:minor encoded)', -COMMENT COLUMN tcp_state 'New TCP state after state change', -COMMENT COLUMN tcp_old_state 'Previous TCP state before state change', -COMMENT COLUMN tcp_srtt_us 'Smoothed RTT in microseconds', -COMMENT COLUMN tcp_cwnd 'Congestion window size', -COMMENT COLUMN exit_code 'Process exit code', -COMMENT COLUMN target_pid 'Target PID for OOM kill events', -COMMENT COLUMN meta_client_name 'Name of the node running the observoor agent', -COMMENT COLUMN meta_network_name 'Ethereum network name (mainnet, holesky, etc.)'; - -CREATE TABLE observoor.raw_events ON CLUSTER '{cluster}' AS observoor.raw_events_local -ENGINE = Distributed('{cluster}', 'observoor', raw_events_local, rand()); - - --------------------------------------------------------------------------------- --- SYNC STATE --- Separate table for consensus/execution layer sync state. --- Polled periodically (e.g., every slot) to reduce per-row overhead. --------------------------------------------------------------------------------- - -CREATE TABLE observoor.sync_state_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - event_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - cl_syncing Bool CODEC(ZSTD(1)), - el_optimistic Bool CODEC(ZSTD(1)), - el_offline Bool CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(event_time) -ORDER BY (meta_network_name, event_time, meta_client_name); - -ALTER TABLE observoor.sync_state_local ON CLUSTER '{cluster}' -MODIFY COMMENT 'Sync state snapshots for consensus and execution layers.', -COMMENT COLUMN updated_date_time 'Version column for ReplacingMergeTree deduplication', -COMMENT COLUMN event_time 'Time when the sync state was sampled', -COMMENT COLUMN wallclock_slot 'Ethereum slot number at sampling time', -COMMENT COLUMN wallclock_slot_start_date_time 'Wall clock time when the slot started', -COMMENT COLUMN cl_syncing 'Whether the consensus layer is syncing', -COMMENT COLUMN el_optimistic 'Whether the execution layer is in optimistic sync mode', -COMMENT COLUMN el_offline 'Whether the execution layer is unreachable', -COMMENT COLUMN meta_client_name 'Name of the node running the observoor agent', -COMMENT COLUMN meta_network_name 'Ethereum network name (mainnet, holesky, etc.)'; - -CREATE TABLE observoor.sync_state ON CLUSTER '{cluster}' AS observoor.sync_state_local -ENGINE = Distributed('{cluster}', 'observoor', sync_state_local, cityHash64(event_time, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- AGGREGATED METRICS: ONE TABLE PER METRIC --- Time-windowed aggregations. No metric_name column - the table IS the metric. --- ReplicatedReplacingMergeTree for idempotent writes, ORDER BY as composite key. --------------------------------------------------------------------------------- - - --------------------------------------------------------------------------------- --- SYSCALL LATENCY TABLES (8 tables) --- Latency histograms for system call operations --------------------------------------------------------------------------------- - -CREATE TABLE observoor.syscall_read_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.syscall_read ON CLUSTER '{cluster}' AS observoor.syscall_read_local -ENGINE = Distributed('{cluster}', 'observoor', syscall_read_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.syscall_write_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.syscall_write ON CLUSTER '{cluster}' AS observoor.syscall_write_local -ENGINE = Distributed('{cluster}', 'observoor', syscall_write_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.syscall_futex_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.syscall_futex ON CLUSTER '{cluster}' AS observoor.syscall_futex_local -ENGINE = Distributed('{cluster}', 'observoor', syscall_futex_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.syscall_mmap_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.syscall_mmap ON CLUSTER '{cluster}' AS observoor.syscall_mmap_local -ENGINE = Distributed('{cluster}', 'observoor', syscall_mmap_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.syscall_epoll_wait_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.syscall_epoll_wait ON CLUSTER '{cluster}' AS observoor.syscall_epoll_wait_local -ENGINE = Distributed('{cluster}', 'observoor', syscall_epoll_wait_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.syscall_fsync_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.syscall_fsync ON CLUSTER '{cluster}' AS observoor.syscall_fsync_local -ENGINE = Distributed('{cluster}', 'observoor', syscall_fsync_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.syscall_fdatasync_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.syscall_fdatasync ON CLUSTER '{cluster}' AS observoor.syscall_fdatasync_local -ENGINE = Distributed('{cluster}', 'observoor', syscall_fdatasync_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.syscall_pwrite_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.syscall_pwrite ON CLUSTER '{cluster}' AS observoor.syscall_pwrite_local -ENGINE = Distributed('{cluster}', 'observoor', syscall_pwrite_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- SCHEDULER LATENCY TABLES (3 tables) --- CPU scheduling latency histograms --------------------------------------------------------------------------------- - -CREATE TABLE observoor.sched_on_cpu_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.sched_on_cpu ON CLUSTER '{cluster}' AS observoor.sched_on_cpu_local -ENGINE = Distributed('{cluster}', 'observoor', sched_on_cpu_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.sched_off_cpu_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.sched_off_cpu ON CLUSTER '{cluster}' AS observoor.sched_off_cpu_local -ENGINE = Distributed('{cluster}', 'observoor', sched_off_cpu_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.sched_runqueue_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.sched_runqueue ON CLUSTER '{cluster}' AS observoor.sched_runqueue_local -ENGINE = Distributed('{cluster}', 'observoor', sched_runqueue_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- MEMORY LATENCY TABLES (2 tables) --- Memory reclaim and compaction latency histograms --------------------------------------------------------------------------------- - -CREATE TABLE observoor.mem_reclaim_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.mem_reclaim ON CLUSTER '{cluster}' AS observoor.mem_reclaim_local -ENGINE = Distributed('{cluster}', 'observoor', mem_reclaim_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.mem_compaction_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.mem_compaction ON CLUSTER '{cluster}' AS observoor.mem_compaction_local -ENGINE = Distributed('{cluster}', 'observoor', mem_compaction_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- DISK LATENCY TABLE (1 table) --- Block I/O latency histogram with device and rw dimensions --------------------------------------------------------------------------------- - -CREATE TABLE observoor.disk_latency_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - device_id UInt32 CODEC(ZSTD(1)), - rw LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - histogram Tuple( - le_1us UInt32, - le_10us UInt32, - le_100us UInt32, - le_1ms UInt32, - le_10ms UInt32, - le_100ms UInt32, - le_1s UInt32, - le_10s UInt32, - le_100s UInt32, - inf UInt32 - ) CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, device_id, rw); - -CREATE TABLE observoor.disk_latency ON CLUSTER '{cluster}' AS observoor.disk_latency_local -ENGINE = Distributed('{cluster}', 'observoor', disk_latency_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- COUNTER TABLES - MEMORY (5 tables) --- Simple count/sum aggregations for memory events --------------------------------------------------------------------------------- - -CREATE TABLE observoor.page_fault_major_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.page_fault_major ON CLUSTER '{cluster}' AS observoor.page_fault_major_local -ENGINE = Distributed('{cluster}', 'observoor', page_fault_major_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.page_fault_minor_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.page_fault_minor ON CLUSTER '{cluster}' AS observoor.page_fault_minor_local -ENGINE = Distributed('{cluster}', 'observoor', page_fault_minor_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.swap_in_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.swap_in ON CLUSTER '{cluster}' AS observoor.swap_in_local -ENGINE = Distributed('{cluster}', 'observoor', swap_in_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.swap_out_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.swap_out ON CLUSTER '{cluster}' AS observoor.swap_out_local -ENGINE = Distributed('{cluster}', 'observoor', swap_out_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.oom_kill_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.oom_kill ON CLUSTER '{cluster}' AS observoor.oom_kill_local -ENGINE = Distributed('{cluster}', 'observoor', oom_kill_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- COUNTER TABLES - PROCESS (3 tables) --- File descriptor and process exit counters --------------------------------------------------------------------------------- - -CREATE TABLE observoor.fd_open_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.fd_open ON CLUSTER '{cluster}' AS observoor.fd_open_local -ENGINE = Distributed('{cluster}', 'observoor', fd_open_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.fd_close_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.fd_close ON CLUSTER '{cluster}' AS observoor.fd_close_local -ENGINE = Distributed('{cluster}', 'observoor', fd_close_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.process_exit_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.process_exit ON CLUSTER '{cluster}' AS observoor.process_exit_local -ENGINE = Distributed('{cluster}', 'observoor', process_exit_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- COUNTER TABLES - NETWORK (3 tables) --- TCP state changes, network I/O, and retransmits --------------------------------------------------------------------------------- - -CREATE TABLE observoor.tcp_state_change_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.tcp_state_change ON CLUSTER '{cluster}' AS observoor.tcp_state_change_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_state_change_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.net_io_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - local_port UInt16 CODEC(ZSTD(1)), - direction LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, local_port, direction); - -CREATE TABLE observoor.net_io ON CLUSTER '{cluster}' AS observoor.net_io_local -ENGINE = Distributed('{cluster}', 'observoor', net_io_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.tcp_retransmit_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - local_port UInt16 CODEC(ZSTD(1)), - direction LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, local_port, direction); - -CREATE TABLE observoor.tcp_retransmit ON CLUSTER '{cluster}' AS observoor.tcp_retransmit_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_retransmit_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- COUNTER TABLES - DISK (2 tables) --- Disk bytes throughput and block merge counters --------------------------------------------------------------------------------- - -CREATE TABLE observoor.disk_bytes_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - device_id UInt32 CODEC(ZSTD(1)), - rw LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, device_id, rw); - -CREATE TABLE observoor.disk_bytes ON CLUSTER '{cluster}' AS observoor.disk_bytes_local -ENGINE = Distributed('{cluster}', 'observoor', disk_bytes_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.block_merge_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - device_id UInt32 CODEC(ZSTD(1)), - rw LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, device_id, rw); - -CREATE TABLE observoor.block_merge ON CLUSTER '{cluster}' AS observoor.block_merge_local -ENGINE = Distributed('{cluster}', 'observoor', block_merge_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- GAUGE TABLES (3 tables) --- Sampled values with min/max/sum/count (no histogram) --------------------------------------------------------------------------------- - -CREATE TABLE observoor.tcp_rtt_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - local_port UInt16 CODEC(ZSTD(1)), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, local_port); - -CREATE TABLE observoor.tcp_rtt ON CLUSTER '{cluster}' AS observoor.tcp_rtt_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_rtt_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.tcp_cwnd_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - local_port UInt16 CODEC(ZSTD(1)), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, local_port); - -CREATE TABLE observoor.tcp_cwnd ON CLUSTER '{cluster}' AS observoor.tcp_cwnd_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_cwnd_local, cityHash64(window_start, meta_network_name, meta_client_name)); - -CREATE TABLE observoor.disk_queue_depth_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - device_id UInt32 CODEC(ZSTD(1)), - rw LowCardinality(String), - sum Int64 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Int64 CODEC(ZSTD(1)), - max Int64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, device_id, rw); - -CREATE TABLE observoor.disk_queue_depth ON CLUSTER '{cluster}' AS observoor.disk_queue_depth_local -ENGINE = Distributed('{cluster}', 'observoor', disk_queue_depth_local, cityHash64(window_start, meta_network_name, meta_client_name)); diff --git a/deploy/migrations/clickhouse/098_observoor_float32_sum_min_max.down.sql b/deploy/migrations/clickhouse/098_observoor_float32_sum_min_max.down.sql deleted file mode 100644 index 7d8ea18f..00000000 --- a/deploy/migrations/clickhouse/098_observoor_float32_sum_min_max.down.sql +++ /dev/null @@ -1,293 +0,0 @@ --- Migration 003 rollback: Float32 → Int64 for sum/min/max columns - --------------------------------------------------------------------------------- --- LATENCY TABLES (14) — restore sum, min, max to Int64 --------------------------------------------------------------------------------- - --- syscall_read -ALTER TABLE observoor.syscall_read_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_read ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- syscall_write -ALTER TABLE observoor.syscall_write_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_write ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- syscall_futex -ALTER TABLE observoor.syscall_futex_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_futex ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- syscall_mmap -ALTER TABLE observoor.syscall_mmap_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_mmap ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- syscall_epoll_wait -ALTER TABLE observoor.syscall_epoll_wait_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_epoll_wait ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- syscall_fsync -ALTER TABLE observoor.syscall_fsync_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_fsync ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- syscall_fdatasync -ALTER TABLE observoor.syscall_fdatasync_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_fdatasync ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- syscall_pwrite -ALTER TABLE observoor.syscall_pwrite_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_pwrite ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- sched_on_cpu -ALTER TABLE observoor.sched_on_cpu_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.sched_on_cpu ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- sched_off_cpu -ALTER TABLE observoor.sched_off_cpu_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.sched_off_cpu ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- sched_runqueue -ALTER TABLE observoor.sched_runqueue_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.sched_runqueue ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- mem_reclaim -ALTER TABLE observoor.mem_reclaim_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.mem_reclaim ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- mem_compaction -ALTER TABLE observoor.mem_compaction_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.mem_compaction ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- disk_latency -ALTER TABLE observoor.disk_latency_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.disk_latency ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - - --------------------------------------------------------------------------------- --- COUNTER TABLES (13) — restore sum to Int64 --------------------------------------------------------------------------------- - --- page_fault_major -ALTER TABLE observoor.page_fault_major_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.page_fault_major ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- page_fault_minor -ALTER TABLE observoor.page_fault_minor_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.page_fault_minor ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- swap_in -ALTER TABLE observoor.swap_in_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.swap_in ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- swap_out -ALTER TABLE observoor.swap_out_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.swap_out ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- oom_kill -ALTER TABLE observoor.oom_kill_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.oom_kill ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- fd_open -ALTER TABLE observoor.fd_open_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.fd_open ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- fd_close -ALTER TABLE observoor.fd_close_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.fd_close ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- process_exit -ALTER TABLE observoor.process_exit_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.process_exit ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- tcp_state_change -ALTER TABLE observoor.tcp_state_change_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.tcp_state_change ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- net_io -ALTER TABLE observoor.net_io_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.net_io ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- tcp_retransmit -ALTER TABLE observoor.tcp_retransmit_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.tcp_retransmit ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- disk_bytes -ALTER TABLE observoor.disk_bytes_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.disk_bytes ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - --- block_merge -ALTER TABLE observoor.block_merge_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.block_merge ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64; - - --------------------------------------------------------------------------------- --- GAUGE TABLES (3) — restore sum, min, max to Int64 --------------------------------------------------------------------------------- - --- tcp_rtt -ALTER TABLE observoor.tcp_rtt_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.tcp_rtt ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- tcp_cwnd -ALTER TABLE observoor.tcp_cwnd_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.tcp_cwnd ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; - --- disk_queue_depth -ALTER TABLE observoor.disk_queue_depth_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Int64 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Int64 CODEC(ZSTD(1)); - -ALTER TABLE observoor.disk_queue_depth ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Int64, - MODIFY COLUMN `min` Int64, - MODIFY COLUMN `max` Int64; diff --git a/deploy/migrations/clickhouse/098_observoor_float32_sum_min_max.up.sql b/deploy/migrations/clickhouse/098_observoor_float32_sum_min_max.up.sql deleted file mode 100644 index b44d8d1e..00000000 --- a/deploy/migrations/clickhouse/098_observoor_float32_sum_min_max.up.sql +++ /dev/null @@ -1,295 +0,0 @@ --- Migration 003: Int64 → Float32 for sum/min/max columns --- Reduces compressed storage ~48% with negligible precision loss for monitoring data. --- ALTER MODIFY COLUMN runs as a background mutation. No downtime required. - --------------------------------------------------------------------------------- --- LATENCY TABLES (14) — modify sum, min, max --------------------------------------------------------------------------------- - --- syscall_read -ALTER TABLE observoor.syscall_read_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_read ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- syscall_write -ALTER TABLE observoor.syscall_write_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_write ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- syscall_futex -ALTER TABLE observoor.syscall_futex_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_futex ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- syscall_mmap -ALTER TABLE observoor.syscall_mmap_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_mmap ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- syscall_epoll_wait -ALTER TABLE observoor.syscall_epoll_wait_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_epoll_wait ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- syscall_fsync -ALTER TABLE observoor.syscall_fsync_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_fsync ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- syscall_fdatasync -ALTER TABLE observoor.syscall_fdatasync_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_fdatasync ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- syscall_pwrite -ALTER TABLE observoor.syscall_pwrite_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.syscall_pwrite ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- sched_on_cpu -ALTER TABLE observoor.sched_on_cpu_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.sched_on_cpu ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- sched_off_cpu -ALTER TABLE observoor.sched_off_cpu_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.sched_off_cpu ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- sched_runqueue -ALTER TABLE observoor.sched_runqueue_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.sched_runqueue ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- mem_reclaim -ALTER TABLE observoor.mem_reclaim_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.mem_reclaim ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- mem_compaction -ALTER TABLE observoor.mem_compaction_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.mem_compaction ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- disk_latency -ALTER TABLE observoor.disk_latency_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.disk_latency ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - - --------------------------------------------------------------------------------- --- COUNTER TABLES (13) — modify sum only --------------------------------------------------------------------------------- - --- page_fault_major -ALTER TABLE observoor.page_fault_major_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.page_fault_major ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- page_fault_minor -ALTER TABLE observoor.page_fault_minor_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.page_fault_minor ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- swap_in -ALTER TABLE observoor.swap_in_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.swap_in ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- swap_out -ALTER TABLE observoor.swap_out_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.swap_out ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- oom_kill -ALTER TABLE observoor.oom_kill_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.oom_kill ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- fd_open -ALTER TABLE observoor.fd_open_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.fd_open ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- fd_close -ALTER TABLE observoor.fd_close_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.fd_close ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- process_exit -ALTER TABLE observoor.process_exit_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.process_exit ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- tcp_state_change -ALTER TABLE observoor.tcp_state_change_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.tcp_state_change ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- net_io -ALTER TABLE observoor.net_io_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.net_io ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- tcp_retransmit -ALTER TABLE observoor.tcp_retransmit_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.tcp_retransmit ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- disk_bytes -ALTER TABLE observoor.disk_bytes_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.disk_bytes ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - --- block_merge -ALTER TABLE observoor.block_merge_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.block_merge ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32; - - --------------------------------------------------------------------------------- --- GAUGE TABLES (3) — modify sum, min, max --------------------------------------------------------------------------------- - --- tcp_rtt -ALTER TABLE observoor.tcp_rtt_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.tcp_rtt ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- tcp_cwnd -ALTER TABLE observoor.tcp_cwnd_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.tcp_cwnd ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; - --- disk_queue_depth -ALTER TABLE observoor.disk_queue_depth_local ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `min` Float32 CODEC(ZSTD(1)), - MODIFY COLUMN `max` Float32 CODEC(ZSTD(1)); - -ALTER TABLE observoor.disk_queue_depth ON CLUSTER '{cluster}' - MODIFY COLUMN `sum` Float32, - MODIFY COLUMN `min` Float32, - MODIFY COLUMN `max` Float32; diff --git a/deploy/migrations/clickhouse/099_observoor_cpu_utilization.down.sql b/deploy/migrations/clickhouse/099_observoor_cpu_utilization.down.sql deleted file mode 100644 index c58d6535..00000000 --- a/deploy/migrations/clickhouse/099_observoor_cpu_utilization.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS observoor.cpu_utilization ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.cpu_utilization_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/099_observoor_cpu_utilization.up.sql b/deploy/migrations/clickhouse/099_observoor_cpu_utilization.up.sql deleted file mode 100644 index f3e97718..00000000 --- a/deploy/migrations/clickhouse/099_observoor_cpu_utilization.up.sql +++ /dev/null @@ -1,36 +0,0 @@ --- Migration 099: add observoor CPU utilization summary table. - -CREATE TABLE observoor.cpu_utilization_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - total_on_cpu_ns Float32 CODEC(ZSTD(1)), - event_count UInt32 CODEC(ZSTD(1)), - active_cores UInt16 CODEC(ZSTD(1)), - system_cores UInt16 CODEC(ZSTD(1)), - max_core_on_cpu_ns Float32 CODEC(ZSTD(1)), - max_core_id UInt32 CODEC(ZSTD(1)), - mean_core_pct Float32 CODEC(ZSTD(1)), - min_core_pct Float32 CODEC(ZSTD(1)), - max_core_pct Float32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.cpu_utilization ON CLUSTER '{cluster}' AS observoor.cpu_utilization_local -ENGINE = Distributed( - '{cluster}', - 'observoor', - cpu_utilization_local, - cityHash64(window_start, meta_network_name, meta_client_name) -); diff --git a/deploy/migrations/clickhouse/100_observoor_port_labels.down.sql b/deploy/migrations/clickhouse/100_observoor_port_labels.down.sql deleted file mode 100644 index 4edabe93..00000000 --- a/deploy/migrations/clickhouse/100_observoor_port_labels.down.sql +++ /dev/null @@ -1,132 +0,0 @@ --- Migration 005 rollback: Restore local_port UInt16 from port_label LowCardinality(String) --- Drops and recreates 4 tables (net_io, tcp_retransmit, tcp_rtt, tcp_cwnd) --- plus their distributed counterparts. Data is lost on rollback. - --------------------------------------------------------------------------------- --- COUNTER TABLES - NETWORK (net_io, tcp_retransmit) --- Restore local_port UInt16 --------------------------------------------------------------------------------- - --- net_io -DROP TABLE IF EXISTS net_io ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS net_io_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE net_io_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - local_port UInt16 CODEC(ZSTD(1)), - direction LowCardinality(String), - sum Float32 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, local_port, direction); - -CREATE TABLE net_io ON CLUSTER '{cluster}' AS net_io_local -ENGINE = Distributed('{cluster}', 'observoor', net_io_local, cityHash64(window_start, meta_network_name, meta_client_name)); - --- tcp_retransmit -DROP TABLE IF EXISTS tcp_retransmit ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tcp_retransmit_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE tcp_retransmit_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - local_port UInt16 CODEC(ZSTD(1)), - direction LowCardinality(String), - sum Float32 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, local_port, direction); - -CREATE TABLE tcp_retransmit ON CLUSTER '{cluster}' AS tcp_retransmit_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_retransmit_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- GAUGE TABLES - TCP (tcp_rtt, tcp_cwnd) --- Restore local_port UInt16 --------------------------------------------------------------------------------- - --- tcp_rtt -DROP TABLE IF EXISTS tcp_rtt ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tcp_rtt_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE tcp_rtt_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - local_port UInt16 CODEC(ZSTD(1)), - sum Float32 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Float32 CODEC(ZSTD(1)), - max Float32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, local_port); - -CREATE TABLE tcp_rtt ON CLUSTER '{cluster}' AS tcp_rtt_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_rtt_local, cityHash64(window_start, meta_network_name, meta_client_name)); - --- tcp_cwnd -DROP TABLE IF EXISTS tcp_cwnd ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS tcp_cwnd_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE tcp_cwnd_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - local_port UInt16 CODEC(ZSTD(1)), - sum Float32 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Float32 CODEC(ZSTD(1)), - max Float32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, local_port); - -CREATE TABLE tcp_cwnd ON CLUSTER '{cluster}' AS tcp_cwnd_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_cwnd_local, cityHash64(window_start, meta_network_name, meta_client_name)); diff --git a/deploy/migrations/clickhouse/100_observoor_port_labels.up.sql b/deploy/migrations/clickhouse/100_observoor_port_labels.up.sql deleted file mode 100644 index 01abe627..00000000 --- a/deploy/migrations/clickhouse/100_observoor_port_labels.up.sql +++ /dev/null @@ -1,133 +0,0 @@ --- Migration 005: Replace local_port UInt16 with port_label LowCardinality(String) --- Drops and recreates 4 tables (net_io, tcp_retransmit, tcp_rtt, tcp_cwnd) --- plus their distributed counterparts. Historical data in these tables is lost. --- local_port is in ORDER BY so ALTER TABLE cannot change the column type. - --------------------------------------------------------------------------------- --- COUNTER TABLES - NETWORK (net_io, tcp_retransmit) --- Replace local_port UInt16 with port_label LowCardinality(String) --------------------------------------------------------------------------------- - --- net_io: drop distributed first, then local -DROP TABLE IF EXISTS observoor.net_io ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS observoor.net_io_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE observoor.net_io_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - port_label LowCardinality(String), - direction LowCardinality(String), - sum Float32 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, port_label, direction); - -CREATE TABLE observoor.net_io ON CLUSTER '{cluster}' AS observoor.net_io_local -ENGINE = Distributed('{cluster}', 'observoor', net_io_local, cityHash64(window_start, meta_network_name, meta_client_name)); - --- tcp_retransmit: drop distributed first, then local -DROP TABLE IF EXISTS observoor.tcp_retransmit ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS observoor.tcp_retransmit_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE observoor.tcp_retransmit_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - port_label LowCardinality(String), - direction LowCardinality(String), - sum Float32 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, port_label, direction); - -CREATE TABLE observoor.tcp_retransmit ON CLUSTER '{cluster}' AS observoor.tcp_retransmit_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_retransmit_local, cityHash64(window_start, meta_network_name, meta_client_name)); - - --------------------------------------------------------------------------------- --- GAUGE TABLES - TCP (tcp_rtt, tcp_cwnd) --- Replace local_port UInt16 with port_label LowCardinality(String) --------------------------------------------------------------------------------- - --- tcp_rtt: drop distributed first, then local -DROP TABLE IF EXISTS observoor.tcp_rtt ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS observoor.tcp_rtt_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE observoor.tcp_rtt_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - port_label LowCardinality(String), - sum Float32 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Float32 CODEC(ZSTD(1)), - max Float32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, port_label); - -CREATE TABLE observoor.tcp_rtt ON CLUSTER '{cluster}' AS observoor.tcp_rtt_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_rtt_local, cityHash64(window_start, meta_network_name, meta_client_name)); - --- tcp_cwnd: drop distributed first, then local -DROP TABLE IF EXISTS observoor.tcp_cwnd ON CLUSTER '{cluster}' SYNC; -DROP TABLE IF EXISTS observoor.tcp_cwnd_local ON CLUSTER '{cluster}' SYNC; - -CREATE TABLE observoor.tcp_cwnd_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - port_label LowCardinality(String), - sum Float32 CODEC(ZSTD(1)), - count UInt32 CODEC(ZSTD(1)), - min Float32 CODEC(ZSTD(1)), - max Float32 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type, port_label); - -CREATE TABLE observoor.tcp_cwnd ON CLUSTER '{cluster}' AS observoor.tcp_cwnd_local -ENGINE = Distributed('{cluster}', 'observoor', tcp_cwnd_local, cityHash64(window_start, meta_network_name, meta_client_name)); diff --git a/deploy/migrations/clickhouse/101_observoor_sampling_metadata.down.sql b/deploy/migrations/clickhouse/101_observoor_sampling_metadata.down.sql deleted file mode 100644 index c0ddd48c..00000000 --- a/deploy/migrations/clickhouse/101_observoor_sampling_metadata.down.sql +++ /dev/null @@ -1,280 +0,0 @@ --- Migration 006 rollback: remove sampling metadata columns from aggregated metric tables. - --- syscall_read -ALTER TABLE observoor.syscall_read_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.syscall_read ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- syscall_write -ALTER TABLE observoor.syscall_write_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.syscall_write ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- syscall_futex -ALTER TABLE observoor.syscall_futex_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.syscall_futex ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- syscall_mmap -ALTER TABLE observoor.syscall_mmap_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.syscall_mmap ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- syscall_epoll_wait -ALTER TABLE observoor.syscall_epoll_wait_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.syscall_epoll_wait ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- syscall_fsync -ALTER TABLE observoor.syscall_fsync_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.syscall_fsync ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- syscall_fdatasync -ALTER TABLE observoor.syscall_fdatasync_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.syscall_fdatasync ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- syscall_pwrite -ALTER TABLE observoor.syscall_pwrite_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.syscall_pwrite ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- sched_on_cpu -ALTER TABLE observoor.sched_on_cpu_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.sched_on_cpu ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- sched_off_cpu -ALTER TABLE observoor.sched_off_cpu_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.sched_off_cpu ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- sched_runqueue -ALTER TABLE observoor.sched_runqueue_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.sched_runqueue ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- mem_reclaim -ALTER TABLE observoor.mem_reclaim_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.mem_reclaim ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- mem_compaction -ALTER TABLE observoor.mem_compaction_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.mem_compaction ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- disk_latency -ALTER TABLE observoor.disk_latency_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.disk_latency ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- page_fault_major -ALTER TABLE observoor.page_fault_major_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.page_fault_major ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- page_fault_minor -ALTER TABLE observoor.page_fault_minor_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.page_fault_minor ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- swap_in -ALTER TABLE observoor.swap_in_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.swap_in ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- swap_out -ALTER TABLE observoor.swap_out_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.swap_out ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- oom_kill -ALTER TABLE observoor.oom_kill_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.oom_kill ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- fd_open -ALTER TABLE observoor.fd_open_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.fd_open ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- fd_close -ALTER TABLE observoor.fd_close_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.fd_close ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- process_exit -ALTER TABLE observoor.process_exit_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.process_exit ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- tcp_state_change -ALTER TABLE observoor.tcp_state_change_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.tcp_state_change ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- net_io -ALTER TABLE observoor.net_io_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.net_io ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- tcp_retransmit -ALTER TABLE observoor.tcp_retransmit_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.tcp_retransmit ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- disk_bytes -ALTER TABLE observoor.disk_bytes_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.disk_bytes ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- block_merge -ALTER TABLE observoor.block_merge_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.block_merge ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- tcp_rtt -ALTER TABLE observoor.tcp_rtt_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.tcp_rtt ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- tcp_cwnd -ALTER TABLE observoor.tcp_cwnd_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.tcp_cwnd ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- disk_queue_depth -ALTER TABLE observoor.disk_queue_depth_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.disk_queue_depth ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - --- cpu_utilization -ALTER TABLE observoor.cpu_utilization_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; - -ALTER TABLE observoor.cpu_utilization ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS sampling_rate, - DROP COLUMN IF EXISTS sampling_mode; diff --git a/deploy/migrations/clickhouse/101_observoor_sampling_metadata.up.sql b/deploy/migrations/clickhouse/101_observoor_sampling_metadata.up.sql deleted file mode 100644 index fcaedbc3..00000000 --- a/deploy/migrations/clickhouse/101_observoor_sampling_metadata.up.sql +++ /dev/null @@ -1,281 +0,0 @@ --- Migration 006: add sampling metadata columns to aggregated metric tables. --- Adds sampling_mode + sampling_rate for downstream extrapolation. - --- syscall_read -ALTER TABLE observoor.syscall_read_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.syscall_read ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- syscall_write -ALTER TABLE observoor.syscall_write_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.syscall_write ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- syscall_futex -ALTER TABLE observoor.syscall_futex_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.syscall_futex ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- syscall_mmap -ALTER TABLE observoor.syscall_mmap_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.syscall_mmap ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- syscall_epoll_wait -ALTER TABLE observoor.syscall_epoll_wait_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.syscall_epoll_wait ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- syscall_fsync -ALTER TABLE observoor.syscall_fsync_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.syscall_fsync ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- syscall_fdatasync -ALTER TABLE observoor.syscall_fdatasync_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.syscall_fdatasync ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- syscall_pwrite -ALTER TABLE observoor.syscall_pwrite_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.syscall_pwrite ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- sched_on_cpu -ALTER TABLE observoor.sched_on_cpu_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.sched_on_cpu ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- sched_off_cpu -ALTER TABLE observoor.sched_off_cpu_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.sched_off_cpu ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- sched_runqueue -ALTER TABLE observoor.sched_runqueue_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.sched_runqueue ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- mem_reclaim -ALTER TABLE observoor.mem_reclaim_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.mem_reclaim ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- mem_compaction -ALTER TABLE observoor.mem_compaction_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.mem_compaction ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- disk_latency -ALTER TABLE observoor.disk_latency_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.disk_latency ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- page_fault_major -ALTER TABLE observoor.page_fault_major_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.page_fault_major ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- page_fault_minor -ALTER TABLE observoor.page_fault_minor_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.page_fault_minor ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- swap_in -ALTER TABLE observoor.swap_in_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.swap_in ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- swap_out -ALTER TABLE observoor.swap_out_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.swap_out ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- oom_kill -ALTER TABLE observoor.oom_kill_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.oom_kill ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- fd_open -ALTER TABLE observoor.fd_open_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.fd_open ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- fd_close -ALTER TABLE observoor.fd_close_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.fd_close ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- process_exit -ALTER TABLE observoor.process_exit_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.process_exit ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- tcp_state_change -ALTER TABLE observoor.tcp_state_change_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.tcp_state_change ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- net_io -ALTER TABLE observoor.net_io_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.net_io ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- tcp_retransmit -ALTER TABLE observoor.tcp_retransmit_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.tcp_retransmit ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- disk_bytes -ALTER TABLE observoor.disk_bytes_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.disk_bytes ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- block_merge -ALTER TABLE observoor.block_merge_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.block_merge ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- tcp_rtt -ALTER TABLE observoor.tcp_rtt_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.tcp_rtt ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- tcp_cwnd -ALTER TABLE observoor.tcp_cwnd_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.tcp_cwnd ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- disk_queue_depth -ALTER TABLE observoor.disk_queue_depth_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.disk_queue_depth ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - --- cpu_utilization -ALTER TABLE observoor.cpu_utilization_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; - -ALTER TABLE observoor.cpu_utilization ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS sampling_mode LowCardinality(String) DEFAULT 'none' AFTER client_type, - ADD COLUMN IF NOT EXISTS sampling_rate Float32 DEFAULT 1.0 AFTER sampling_mode; diff --git a/deploy/migrations/clickhouse/102_execution_block_metrics.down.sql b/deploy/migrations/clickhouse/102_execution_block_metrics.down.sql deleted file mode 100644 index a9f922e7..00000000 --- a/deploy/migrations/clickhouse/102_execution_block_metrics.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS execution_block_metrics ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS execution_block_metrics_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/102_execution_block_metrics.up.sql b/deploy/migrations/clickhouse/102_execution_block_metrics.up.sql deleted file mode 100644 index 3e8bf0a7..00000000 --- a/deploy/migrations/clickhouse/102_execution_block_metrics.up.sql +++ /dev/null @@ -1,89 +0,0 @@ -CREATE TABLE execution_block_metrics_local ON CLUSTER '{cluster}' ( - -- Timestamps - updated_date_time DateTime COMMENT 'Timestamp when the record was last updated' Codec(DoubleDelta, ZSTD(1)), - event_date_time DateTime64(3) COMMENT 'When the event was received' Codec(DoubleDelta, ZSTD(1)), - - -- Source tracking - source LowCardinality(String) COMMENT 'Data source (e.g., client-logs)', - - -- Block info - block_number UInt64 COMMENT 'Execution block number' Codec(DoubleDelta, ZSTD(1)), - block_hash FixedString(66) COMMENT 'Execution block hash (hex encoded with 0x prefix)' Codec(ZSTD(1)), - gas_used UInt64 COMMENT 'Total gas used by all transactions in the block' Codec(ZSTD(1)), - tx_count UInt32 COMMENT 'Number of transactions in the block' Codec(ZSTD(1)), - - -- Timing in milliseconds - execution_ms Float64 COMMENT 'Time spent executing transactions in milliseconds' Codec(ZSTD(1)), - state_read_ms Float64 COMMENT 'Time spent reading state in milliseconds' Codec(ZSTD(1)), - state_hash_ms Float64 COMMENT 'Time spent computing state hash in milliseconds' Codec(ZSTD(1)), - commit_ms Float64 COMMENT 'Time spent committing state changes in milliseconds' Codec(ZSTD(1)), - total_ms Float64 COMMENT 'Total time for block processing in milliseconds' Codec(ZSTD(1)), - - -- Throughput - mgas_per_sec Float64 COMMENT 'Throughput in million gas per second' Codec(ZSTD(1)), - - -- State reads - state_reads_accounts UInt64 COMMENT 'Number of account reads' Codec(ZSTD(1)), - state_reads_storage_slots UInt64 COMMENT 'Number of storage slot reads' Codec(ZSTD(1)), - state_reads_code UInt64 COMMENT 'Number of code reads' Codec(ZSTD(1)), - state_reads_code_bytes UInt64 COMMENT 'Total bytes of code read' Codec(ZSTD(1)), - - -- State writes - state_writes_accounts UInt64 COMMENT 'Number of account writes' Codec(ZSTD(1)), - state_writes_accounts_deleted UInt64 COMMENT 'Number of accounts deleted' Codec(ZSTD(1)), - state_writes_storage_slots UInt64 COMMENT 'Number of storage slot writes' Codec(ZSTD(1)), - state_writes_storage_slots_deleted UInt64 COMMENT 'Number of storage slots deleted' Codec(ZSTD(1)), - state_writes_code UInt64 COMMENT 'Number of code writes' Codec(ZSTD(1)), - state_writes_code_bytes UInt64 COMMENT 'Total bytes of code written' Codec(ZSTD(1)), - - -- Cache metrics - account_cache_hits Int64 COMMENT 'Number of account cache hits' Codec(ZSTD(1)), - account_cache_misses Int64 COMMENT 'Number of account cache misses' Codec(ZSTD(1)), - account_cache_hit_rate Float64 COMMENT 'Account cache hit rate as percentage' Codec(ZSTD(1)), - storage_cache_hits Int64 COMMENT 'Number of storage cache hits' Codec(ZSTD(1)), - storage_cache_misses Int64 COMMENT 'Number of storage cache misses' Codec(ZSTD(1)), - storage_cache_hit_rate Float64 COMMENT 'Storage cache hit rate as percentage' Codec(ZSTD(1)), - code_cache_hits Int64 COMMENT 'Number of code cache hits' Codec(ZSTD(1)), - code_cache_misses Int64 COMMENT 'Number of code cache misses' Codec(ZSTD(1)), - code_cache_hit_rate Float64 COMMENT 'Code cache hit rate as percentage' Codec(ZSTD(1)), - code_cache_hit_bytes Int64 COMMENT 'Total bytes of code cache hits' Codec(ZSTD(1)), - code_cache_miss_bytes Int64 COMMENT 'Total bytes of code cache misses' Codec(ZSTD(1)), - - -- Standard metadata fields - meta_client_name LowCardinality(String) COMMENT 'Name of the client that generated the event', - meta_client_id String COMMENT 'Unique Session ID of the client that generated the event' Codec(ZSTD(1)), - meta_client_version LowCardinality(String) COMMENT 'Version of the client that generated the event', - meta_client_implementation LowCardinality(String) COMMENT 'Implementation of the client that generated the event', - meta_client_os LowCardinality(String) COMMENT 'Operating system of the client that generated the event', - meta_client_ip Nullable(IPv6) COMMENT 'IP address of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_city LowCardinality(String) COMMENT 'City of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country LowCardinality(String) COMMENT 'Country of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_country_code LowCardinality(String) COMMENT 'Country code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_continent_code LowCardinality(String) COMMENT 'Continent code of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_longitude Nullable(Float64) COMMENT 'Longitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_latitude Nullable(Float64) COMMENT 'Latitude of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_number Nullable(UInt32) COMMENT 'Autonomous system number of the client that generated the event' Codec(ZSTD(1)), - meta_client_geo_autonomous_system_organization Nullable(String) COMMENT 'Autonomous system organization of the client that generated the event' Codec(ZSTD(1)), - meta_network_id Int32 COMMENT 'Ethereum network ID' Codec(DoubleDelta, ZSTD(1)), - meta_network_name LowCardinality(String) COMMENT 'Ethereum network name', - meta_labels Map(String, String) COMMENT 'Labels associated with the event' Codec(ZSTD(1)) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY intDiv(block_number, 5000000) -ORDER BY (block_number, meta_network_name, meta_client_name, event_date_time) -COMMENT 'Contains detailed performance metrics from execution client structured logging for block execution'; - -CREATE TABLE execution_block_metrics ON CLUSTER '{cluster}' AS execution_block_metrics_local -ENGINE = Distributed( - '{cluster}', - default, - execution_block_metrics_local, - cityHash64( - block_number, - meta_network_name, - meta_client_name - ) -); diff --git a/deploy/migrations/clickhouse/103_observoor_process_snapshots.down.sql b/deploy/migrations/clickhouse/103_observoor_process_snapshots.down.sql deleted file mode 100644 index 2c432cd7..00000000 --- a/deploy/migrations/clickhouse/103_observoor_process_snapshots.down.sql +++ /dev/null @@ -1,14 +0,0 @@ -DROP TABLE IF EXISTS observoor.host_specs ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.host_specs_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS observoor.process_sched_usage ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.process_sched_usage_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS observoor.process_fd_usage ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.process_fd_usage_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS observoor.process_io_usage ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.process_io_usage_local ON CLUSTER '{cluster}'; - -DROP TABLE IF EXISTS observoor.memory_usage ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS observoor.memory_usage_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/103_observoor_process_snapshots.up.sql b/deploy/migrations/clickhouse/103_observoor_process_snapshots.up.sql deleted file mode 100644 index fa390218..00000000 --- a/deploy/migrations/clickhouse/103_observoor_process_snapshots.up.sql +++ /dev/null @@ -1,198 +0,0 @@ --- Migration 103: add process snapshot tables and periodic host_specs snapshots. - -CREATE TABLE observoor.memory_usage_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sampling_mode LowCardinality(String), - sampling_rate Float32 CODEC(ZSTD(1)), - vm_size_bytes UInt64 CODEC(ZSTD(1)), - vm_rss_bytes UInt64 CODEC(ZSTD(1)), - rss_anon_bytes UInt64 CODEC(ZSTD(1)), - rss_file_bytes UInt64 CODEC(ZSTD(1)), - rss_shmem_bytes UInt64 CODEC(ZSTD(1)), - vm_swap_bytes UInt64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.memory_usage ON CLUSTER '{cluster}' AS observoor.memory_usage_local -ENGINE = Distributed( - '{cluster}', - 'observoor', - memory_usage_local, - cityHash64(window_start, meta_network_name, meta_client_name) -); - -CREATE TABLE observoor.process_io_usage_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sampling_mode LowCardinality(String), - sampling_rate Float32 CODEC(ZSTD(1)), - rchar_bytes UInt64 CODEC(ZSTD(1)), - wchar_bytes UInt64 CODEC(ZSTD(1)), - syscr UInt64 CODEC(ZSTD(1)), - syscw UInt64 CODEC(ZSTD(1)), - read_bytes UInt64 CODEC(ZSTD(1)), - write_bytes UInt64 CODEC(ZSTD(1)), - cancelled_write_bytes Int64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.process_io_usage ON CLUSTER '{cluster}' AS observoor.process_io_usage_local -ENGINE = Distributed( - '{cluster}', - 'observoor', - process_io_usage_local, - cityHash64(window_start, meta_network_name, meta_client_name) -); - -CREATE TABLE observoor.process_fd_usage_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sampling_mode LowCardinality(String), - sampling_rate Float32 CODEC(ZSTD(1)), - open_fds UInt32 CODEC(ZSTD(1)), - fd_limit_soft UInt64 CODEC(ZSTD(1)), - fd_limit_hard UInt64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.process_fd_usage ON CLUSTER '{cluster}' AS observoor.process_fd_usage_local -ENGINE = Distributed( - '{cluster}', - 'observoor', - process_fd_usage_local, - cityHash64(window_start, meta_network_name, meta_client_name) -); - -CREATE TABLE observoor.process_sched_usage_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - window_start DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - interval_ms UInt16 CODEC(ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - pid UInt32 CODEC(ZSTD(1)), - client_type LowCardinality(String), - sampling_mode LowCardinality(String), - sampling_rate Float32 CODEC(ZSTD(1)), - threads UInt32 CODEC(ZSTD(1)), - voluntary_ctxt_switches UInt64 CODEC(ZSTD(1)), - nonvoluntary_ctxt_switches UInt64 CODEC(ZSTD(1)), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(window_start) -ORDER BY (meta_network_name, window_start, meta_client_name, pid, client_type); - -CREATE TABLE observoor.process_sched_usage ON CLUSTER '{cluster}' AS observoor.process_sched_usage_local -ENGINE = Distributed( - '{cluster}', - 'observoor', - process_sched_usage_local, - cityHash64(window_start, meta_network_name, meta_client_name) -); - -CREATE TABLE observoor.host_specs_local ON CLUSTER '{cluster}' ( - updated_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - event_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot UInt32 CODEC(DoubleDelta, ZSTD(1)), - wallclock_slot_start_date_time DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - host_id String, - kernel_release LowCardinality(String), - os_name LowCardinality(String), - architecture LowCardinality(String), - cpu_model String, - cpu_vendor LowCardinality(String), - cpu_online_cores UInt16 CODEC(ZSTD(1)), - cpu_logical_cores UInt16 CODEC(ZSTD(1)), - cpu_physical_cores UInt16 CODEC(ZSTD(1)), - cpu_performance_cores UInt16 CODEC(ZSTD(1)), - cpu_efficiency_cores UInt16 CODEC(ZSTD(1)), - cpu_unknown_type_cores UInt16 CODEC(ZSTD(1)), - cpu_logical_ids Array(UInt16), - cpu_core_ids Array(Int32), - cpu_package_ids Array(Int32), - cpu_die_ids Array(Int32), - cpu_cluster_ids Array(Int32), - cpu_core_types Array(UInt8), - cpu_core_type_labels Array(String), - cpu_online_flags Array(UInt8), - cpu_max_freq_khz Array(UInt64), - cpu_base_freq_khz Array(UInt64), - memory_total_bytes UInt64 CODEC(ZSTD(1)), - memory_type LowCardinality(String), - memory_speed_mts UInt32 CODEC(ZSTD(1)), - memory_dimm_count UInt16 CODEC(ZSTD(1)), - memory_dimm_sizes_bytes Array(UInt64), - memory_dimm_types Array(String), - memory_dimm_speeds_mts Array(UInt32), - memory_dimm_configured_speeds_mts Array(UInt32), - memory_dimm_locators Array(String), - memory_dimm_bank_locators Array(String), - memory_dimm_manufacturers Array(String), - memory_dimm_part_numbers Array(String), - memory_dimm_serials Array(String), - disk_count UInt16 CODEC(ZSTD(1)), - disk_total_bytes UInt64 CODEC(ZSTD(1)), - disk_names Array(String), - disk_models Array(String), - disk_vendors Array(String), - disk_serials Array(String), - disk_sizes_bytes Array(UInt64), - disk_rotational Array(UInt8), - meta_client_name LowCardinality(String), - meta_network_name LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', - '{replica}', - updated_date_time -) -PARTITION BY toStartOfMonth(event_time) -ORDER BY (meta_network_name, event_time, host_id, meta_client_name); - -CREATE TABLE observoor.host_specs ON CLUSTER '{cluster}' AS observoor.host_specs_local -ENGINE = Distributed( - '{cluster}', - 'observoor', - host_specs_local, - cityHash64(event_time, meta_network_name, host_id, meta_client_name) -); diff --git a/deploy/migrations/clickhouse/104_structlog_agg_resource_gas_columns.down.sql b/deploy/migrations/clickhouse/104_structlog_agg_resource_gas_columns.down.sql deleted file mode 100644 index 2b85dc51..00000000 --- a/deploy/migrations/clickhouse/104_structlog_agg_resource_gas_columns.down.sql +++ /dev/null @@ -1,15 +0,0 @@ -ALTER TABLE canonical_execution_transaction_structlog_agg_local ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS `memory_words_sum_before`, - DROP COLUMN IF EXISTS `memory_words_sum_after`, - DROP COLUMN IF EXISTS `memory_words_sq_sum_before`, - DROP COLUMN IF EXISTS `memory_words_sq_sum_after`, - DROP COLUMN IF EXISTS `memory_expansion_gas`, - DROP COLUMN IF EXISTS `cold_access_count`; - -ALTER TABLE canonical_execution_transaction_structlog_agg ON CLUSTER '{cluster}' - DROP COLUMN IF EXISTS `memory_words_sum_before`, - DROP COLUMN IF EXISTS `memory_words_sum_after`, - DROP COLUMN IF EXISTS `memory_words_sq_sum_before`, - DROP COLUMN IF EXISTS `memory_words_sq_sum_after`, - DROP COLUMN IF EXISTS `memory_expansion_gas`, - DROP COLUMN IF EXISTS `cold_access_count`; diff --git a/deploy/migrations/clickhouse/104_structlog_agg_resource_gas_columns.up.sql b/deploy/migrations/clickhouse/104_structlog_agg_resource_gas_columns.up.sql deleted file mode 100644 index c8ff5fb6..00000000 --- a/deploy/migrations/clickhouse/104_structlog_agg_resource_gas_columns.up.sql +++ /dev/null @@ -1,19 +0,0 @@ --- Add resource gas building block columns for decomposing EVM gas into categories. --- These columns enable downstream SQL to compute memory expansion gas and cold access gas --- without needing per-opcode structlog data. - -ALTER TABLE canonical_execution_transaction_structlog_agg_local ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS `memory_words_sum_before` UInt64 DEFAULT 0 COMMENT 'SUM(ceil(memory_bytes/32)) before each opcode executes. Used with sq_sum to compute memory expansion gas.' CODEC(ZSTD(1)) AFTER `max_depth`, - ADD COLUMN IF NOT EXISTS `memory_words_sum_after` UInt64 DEFAULT 0 COMMENT 'SUM(ceil(memory_bytes/32)) after each opcode executes.' CODEC(ZSTD(1)) AFTER `memory_words_sum_before`, - ADD COLUMN IF NOT EXISTS `memory_words_sq_sum_before` UInt64 DEFAULT 0 COMMENT 'SUM(words_before²). With sum_before, enables exact memory gas via E[cost(after)] - E[cost(before)].' CODEC(ZSTD(1)) AFTER `memory_words_sum_after`, - ADD COLUMN IF NOT EXISTS `memory_words_sq_sum_after` UInt64 DEFAULT 0 COMMENT 'SUM(words_after²). With sum_after, enables exact memory gas via E[cost(after)] - E[cost(before)].' CODEC(ZSTD(1)) AFTER `memory_words_sq_sum_before`, - ADD COLUMN IF NOT EXISTS `memory_expansion_gas` UInt64 DEFAULT 0 COMMENT 'SUM(memory_expansion_gas). Exact per-opcode memory expansion cost, pre-computed to avoid intDiv rounding in SQL reconstruction.' CODEC(ZSTD(1)) AFTER `memory_words_sq_sum_after`, - ADD COLUMN IF NOT EXISTS `cold_access_count` UInt64 DEFAULT 0 COMMENT 'Number of cold storage/account accesses (EIP-2929). cold_gas = cold_count * (cold_cost - warm_cost).' CODEC(ZSTD(1)) AFTER `memory_expansion_gas`; - -ALTER TABLE canonical_execution_transaction_structlog_agg ON CLUSTER '{cluster}' - ADD COLUMN IF NOT EXISTS `memory_words_sum_before` UInt64 DEFAULT 0 COMMENT 'SUM(ceil(memory_bytes/32)) before each opcode executes.' AFTER `max_depth`, - ADD COLUMN IF NOT EXISTS `memory_words_sum_after` UInt64 DEFAULT 0 COMMENT 'SUM(ceil(memory_bytes/32)) after each opcode executes.' AFTER `memory_words_sum_before`, - ADD COLUMN IF NOT EXISTS `memory_words_sq_sum_before` UInt64 DEFAULT 0 COMMENT 'SUM(words_before²).' AFTER `memory_words_sum_after`, - ADD COLUMN IF NOT EXISTS `memory_words_sq_sum_after` UInt64 DEFAULT 0 COMMENT 'SUM(words_after²).' AFTER `memory_words_sq_sum_before`, - ADD COLUMN IF NOT EXISTS `memory_expansion_gas` UInt64 DEFAULT 0 COMMENT 'SUM(memory_expansion_gas). Exact per-opcode memory expansion cost.' AFTER `memory_words_sq_sum_after`, - ADD COLUMN IF NOT EXISTS `cold_access_count` UInt64 DEFAULT 0 COMMENT 'Number of cold storage/account accesses (EIP-2929).' AFTER `memory_expansion_gas`; diff --git a/deploy/migrations/clickhouse/105_libp2p_identify.down.sql b/deploy/migrations/clickhouse/105_libp2p_identify.down.sql deleted file mode 100644 index b4b6d7fc..00000000 --- a/deploy/migrations/clickhouse/105_libp2p_identify.down.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS libp2p_identify ON CLUSTER '{cluster}'; -DROP TABLE IF EXISTS libp2p_identify_local ON CLUSTER '{cluster}'; diff --git a/deploy/migrations/clickhouse/105_libp2p_identify.up.sql b/deploy/migrations/clickhouse/105_libp2p_identify.up.sql deleted file mode 100644 index 5ed4fbc4..00000000 --- a/deploy/migrations/clickhouse/105_libp2p_identify.up.sql +++ /dev/null @@ -1,65 +0,0 @@ -CREATE TABLE IF NOT EXISTS libp2p_identify_local ON CLUSTER '{cluster}' ( - `updated_date_time` DateTime CODEC(DoubleDelta, ZSTD(1)), - `event_date_time` DateTime64(3) CODEC(DoubleDelta, ZSTD(1)), - `remote_peer_id_unique_key` Int64 CODEC(ZSTD(1)), - `success` Bool CODEC(ZSTD(1)), - `error` Nullable(String) CODEC(ZSTD(1)), - `remote_protocol` LowCardinality(String), - `remote_transport_protocol` LowCardinality(String), - `remote_port` UInt16 CODEC(ZSTD(1)), - `remote_ip` Nullable(IPv6) CODEC(ZSTD(1)), - `remote_geo_city` LowCardinality(String) CODEC(ZSTD(1)), - `remote_geo_country` LowCardinality(String) CODEC(ZSTD(1)), - `remote_geo_country_code` LowCardinality(String) CODEC(ZSTD(1)), - `remote_geo_continent_code` LowCardinality(String) CODEC(ZSTD(1)), - `remote_geo_longitude` Nullable(Float64) CODEC(ZSTD(1)), - `remote_geo_latitude` Nullable(Float64) CODEC(ZSTD(1)), - `remote_geo_autonomous_system_number` Nullable(UInt32) CODEC(ZSTD(1)), - `remote_geo_autonomous_system_organization` Nullable(String) CODEC(ZSTD(1)), - `remote_agent_implementation` LowCardinality(String), - `remote_agent_version` LowCardinality(String), - `remote_agent_version_major` LowCardinality(String), - `remote_agent_version_minor` LowCardinality(String), - `remote_agent_version_patch` LowCardinality(String), - `remote_agent_platform` LowCardinality(String), - `protocol_version` LowCardinality(String), - `protocols` Array(String) CODEC(ZSTD(1)), - `listen_addrs` Array(String) CODEC(ZSTD(1)), - `observed_addr` String CODEC(ZSTD(1)), - `transport` LowCardinality(String), - `security` LowCardinality(String), - `muxer` LowCardinality(String), - `direction` LowCardinality(String), - `remote_multiaddr` String CODEC(ZSTD(1)), - `meta_client_name` LowCardinality(String), - `meta_client_id` String CODEC(ZSTD(1)), - `meta_client_version` LowCardinality(String), - `meta_client_implementation` LowCardinality(String), - `meta_client_os` LowCardinality(String), - `meta_client_ip` Nullable(IPv6) CODEC(ZSTD(1)), - `meta_client_geo_city` LowCardinality(String) CODEC(ZSTD(1)), - `meta_client_geo_country` LowCardinality(String) CODEC(ZSTD(1)), - `meta_client_geo_country_code` LowCardinality(String) CODEC(ZSTD(1)), - `meta_client_geo_continent_code` LowCardinality(String) CODEC(ZSTD(1)), - `meta_client_geo_longitude` Nullable(Float64) CODEC(ZSTD(1)), - `meta_client_geo_latitude` Nullable(Float64) CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_number` Nullable(UInt32) CODEC(ZSTD(1)), - `meta_client_geo_autonomous_system_organization` Nullable(String) CODEC(ZSTD(1)), - `meta_network_id` Int32 CODEC(DoubleDelta, ZSTD(1)), - `meta_network_name` LowCardinality(String) -) ENGINE = ReplicatedReplacingMergeTree( - '/clickhouse/{installation}/{cluster}/default/tables/libp2p_identify_local/{shard}', - '{replica}', - updated_date_time -) -PARTITION BY toYYYYMM(event_date_time) -ORDER BY (event_date_time, meta_network_name, meta_client_name, remote_peer_id_unique_key, direction) -SETTINGS index_granularity = 8192; - -CREATE TABLE IF NOT EXISTS libp2p_identify ON CLUSTER '{cluster}' AS libp2p_identify_local -ENGINE = Distributed( - '{cluster}', - 'default', - 'libp2p_identify_local', - cityHash64(event_date_time, meta_network_name, meta_client_name, remote_peer_id_unique_key, direction) -); diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go index 26e65126..d4f6de75 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.gen.go @@ -13,6 +13,7 @@ import ( const beaconApiEthV1BeaconBlobTableName route.TableName = "beacon_api_eth_v1_beacon_blob" type beaconApiEthV1BeaconBlobBatch struct { + UpdatedDateTime proto.ColDateTime EventDateTime proto.ColDateTime64 Slot proto.ColUInt32 SlotStartDateTime proto.ColDateTime @@ -25,7 +26,6 @@ type beaconApiEthV1BeaconBlobBatch struct { KzgCommitment route.SafeColFixedStr VersionedHash route.SafeColFixedStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -38,14 +38,12 @@ type beaconApiEthV1BeaconBlobBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -61,7 +59,6 @@ func newbeaconApiEthV1BeaconBlobBatch() *beaconApiEthV1BeaconBlobBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -72,7 +69,6 @@ func (b *beaconApiEthV1BeaconBlobBatch) Rows() int { func (b *beaconApiEthV1BeaconBlobBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -85,19 +81,16 @@ func (b *beaconApiEthV1BeaconBlobBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -110,7 +103,6 @@ func (b *beaconApiEthV1BeaconBlobBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -118,15 +110,11 @@ func (b *beaconApiEthV1BeaconBlobBatch) appendMetadata(event *xatu.DecoratedEven b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1BeaconBlobBatch) Input() proto.Input { return proto.Input{ + {Name: "updated_date_time", Data: &b.UpdatedDateTime}, {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "slot", Data: &b.Slot}, {Name: "slot_start_date_time", Data: &b.SlotStartDateTime}, @@ -139,7 +127,6 @@ func (b *beaconApiEthV1BeaconBlobBatch) Input() proto.Input { {Name: "kzg_commitment", Data: &b.KzgCommitment}, {Name: "versioned_hash", Data: &b.VersionedHash}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -152,18 +139,17 @@ func (b *beaconApiEthV1BeaconBlobBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } func (b *beaconApiEthV1BeaconBlobBatch) Reset() { + b.UpdatedDateTime.Reset() b.EventDateTime.Reset() b.Slot.Reset() b.SlotStartDateTime.Reset() @@ -176,7 +162,6 @@ func (b *beaconApiEthV1BeaconBlobBatch) Reset() { b.KzgCommitment.Reset() b.VersionedHash.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -189,14 +174,12 @@ func (b *beaconApiEthV1BeaconBlobBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -205,7 +188,8 @@ func (b *beaconApiEthV1BeaconBlobBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 33) + row := make(map[string]any, 31) + row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -218,7 +202,6 @@ func (b *beaconApiEthV1BeaconBlobBatch) Snapshot() []map[string]any { row["kzg_commitment"] = string(b.KzgCommitment.Row(i)) row["versioned_hash"] = string(b.VersionedHash.Row(i)) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -251,14 +234,12 @@ func (b *beaconApiEthV1BeaconBlobBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.go index 9b1f4894..f4baae6b 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_blob.go @@ -70,6 +70,8 @@ func (b *beaconApiEthV1BeaconBlobBatch) validate(event *xatu.DecoratedEvent) err } func (b *beaconApiEthV1BeaconBlobBatch) appendRuntime(event *xatu.DecoratedEvent) { + b.UpdatedDateTime.Append(time.Now()) + if ts := event.GetEvent().GetDateTime(); ts != nil { b.EventDateTime.Append(ts.AsTime()) } else { diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go index 9ec80ba1..42eb5ebf 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_beacon_committee.gen.go @@ -22,7 +22,6 @@ type beaconApiEthV1BeaconCommitteeBatch struct { Epoch proto.ColUInt32 EpochStartDateTime proto.ColDateTime MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -35,14 +34,12 @@ type beaconApiEthV1BeaconCommitteeBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -55,7 +52,6 @@ func newbeaconApiEthV1BeaconCommitteeBatch() *beaconApiEthV1BeaconCommitteeBatch MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -66,7 +62,6 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) Rows() int { func (b *beaconApiEthV1BeaconCommitteeBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -79,19 +74,16 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) appendMetadata(event *xatu.Decorate b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -104,7 +96,6 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) appendMetadata(event *xatu.Decorate b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -112,11 +103,6 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) appendMetadata(event *xatu.Decorate b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1BeaconCommitteeBatch) Input() proto.Input { @@ -130,7 +116,6 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) Input() proto.Input { {Name: "epoch", Data: &b.Epoch}, {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -143,14 +128,12 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -164,7 +147,6 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) Reset() { b.Epoch.Reset() b.EpochStartDateTime.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -177,14 +159,12 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -193,7 +173,7 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 30) + row := make(map[string]any, 27) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -203,7 +183,6 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) Snapshot() []map[string]any { row["epoch"] = b.Epoch.Row(i) row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -236,14 +215,12 @@ func (b *beaconApiEthV1BeaconCommitteeBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go index 6c6411e6..6343d6c1 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.gen.go @@ -13,6 +13,7 @@ import ( const beaconApiEthV1EventsAttestationTableName route.TableName = "beacon_api_eth_v1_events_attestation" type beaconApiEthV1EventsAttestationBatch struct { + UpdatedDateTime proto.ColDateTime EventDateTime proto.ColDateTime64 Slot proto.ColUInt32 SlotStartDateTime proto.ColDateTime @@ -31,7 +32,6 @@ type beaconApiEthV1EventsAttestationBatch struct { TargetEpochStartDateTime proto.ColDateTime TargetRoot route.SafeColFixedStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -44,14 +44,12 @@ type beaconApiEthV1EventsAttestationBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -67,7 +65,6 @@ func newbeaconApiEthV1EventsAttestationBatch() *beaconApiEthV1EventsAttestationB MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -78,7 +75,6 @@ func (b *beaconApiEthV1EventsAttestationBatch) Rows() int { func (b *beaconApiEthV1EventsAttestationBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -91,19 +87,16 @@ func (b *beaconApiEthV1EventsAttestationBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -116,7 +109,6 @@ func (b *beaconApiEthV1EventsAttestationBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -124,15 +116,11 @@ func (b *beaconApiEthV1EventsAttestationBatch) appendMetadata(event *xatu.Decora b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsAttestationBatch) Input() proto.Input { return proto.Input{ + {Name: "updated_date_time", Data: &b.UpdatedDateTime}, {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "slot", Data: &b.Slot}, {Name: "slot_start_date_time", Data: &b.SlotStartDateTime}, @@ -151,7 +139,6 @@ func (b *beaconApiEthV1EventsAttestationBatch) Input() proto.Input { {Name: "target_epoch_start_date_time", Data: &b.TargetEpochStartDateTime}, {Name: "target_root", Data: &b.TargetRoot}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -164,18 +151,17 @@ func (b *beaconApiEthV1EventsAttestationBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } func (b *beaconApiEthV1EventsAttestationBatch) Reset() { + b.UpdatedDateTime.Reset() b.EventDateTime.Reset() b.Slot.Reset() b.SlotStartDateTime.Reset() @@ -194,7 +180,6 @@ func (b *beaconApiEthV1EventsAttestationBatch) Reset() { b.TargetEpochStartDateTime.Reset() b.TargetRoot.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -207,14 +192,12 @@ func (b *beaconApiEthV1EventsAttestationBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -223,7 +206,8 @@ func (b *beaconApiEthV1EventsAttestationBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 39) + row := make(map[string]any, 37) + row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -246,7 +230,6 @@ func (b *beaconApiEthV1EventsAttestationBatch) Snapshot() []map[string]any { row["target_epoch_start_date_time"] = b.TargetEpochStartDateTime.Row(i).Unix() row["target_root"] = string(b.TargetRoot.Row(i)) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -279,14 +262,12 @@ func (b *beaconApiEthV1EventsAttestationBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.go index c2a05559..977dd219 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_attestation.go @@ -81,6 +81,8 @@ func (b *beaconApiEthV1EventsAttestationBatch) validate(event *xatu.DecoratedEve } func (b *beaconApiEthV1EventsAttestationBatch) appendRuntime(event *xatu.DecoratedEvent) { + b.UpdatedDateTime.Append(time.Now()) + if ts := event.GetEvent().GetDateTime(); ts != nil { b.EventDateTime.Append(ts.AsTime()) } else { diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go index ce023fe4..9e9767eb 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_blob_sidecar.gen.go @@ -25,7 +25,6 @@ type beaconApiEthV1EventsBlobSidecarBatch struct { KzgCommitment route.SafeColFixedStr VersionedHash route.SafeColFixedStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -38,14 +37,12 @@ type beaconApiEthV1EventsBlobSidecarBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -60,7 +57,6 @@ func newbeaconApiEthV1EventsBlobSidecarBatch() *beaconApiEthV1EventsBlobSidecarB MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -71,7 +67,6 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) Rows() int { func (b *beaconApiEthV1EventsBlobSidecarBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -84,19 +79,16 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -109,7 +101,6 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -117,11 +108,6 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) appendMetadata(event *xatu.Decora b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsBlobSidecarBatch) Input() proto.Input { @@ -138,7 +124,6 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) Input() proto.Input { {Name: "kzg_commitment", Data: &b.KzgCommitment}, {Name: "versioned_hash", Data: &b.VersionedHash}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -151,14 +136,12 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -175,7 +158,6 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) Reset() { b.KzgCommitment.Reset() b.VersionedHash.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -188,14 +170,12 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -204,7 +184,7 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 33) + row := make(map[string]any, 30) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -217,7 +197,6 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) Snapshot() []map[string]any { row["kzg_commitment"] = string(b.KzgCommitment.Row(i)) row["versioned_hash"] = string(b.VersionedHash.Row(i)) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -250,14 +229,12 @@ func (b *beaconApiEthV1EventsBlobSidecarBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go index 68e3a1ac..aff79986 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block.gen.go @@ -23,7 +23,6 @@ type beaconApiEthV1EventsBlockBatch struct { EpochStartDateTime proto.ColDateTime ExecutionOptimistic proto.ColBool MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -36,14 +35,12 @@ type beaconApiEthV1EventsBlockBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -56,7 +53,6 @@ func newbeaconApiEthV1EventsBlockBatch() *beaconApiEthV1EventsBlockBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -67,7 +63,6 @@ func (b *beaconApiEthV1EventsBlockBatch) Rows() int { func (b *beaconApiEthV1EventsBlockBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -80,19 +75,16 @@ func (b *beaconApiEthV1EventsBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -105,7 +97,6 @@ func (b *beaconApiEthV1EventsBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -113,11 +104,6 @@ func (b *beaconApiEthV1EventsBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsBlockBatch) Input() proto.Input { @@ -132,7 +118,6 @@ func (b *beaconApiEthV1EventsBlockBatch) Input() proto.Input { {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "execution_optimistic", Data: &b.ExecutionOptimistic}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -145,14 +130,12 @@ func (b *beaconApiEthV1EventsBlockBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -167,7 +150,6 @@ func (b *beaconApiEthV1EventsBlockBatch) Reset() { b.EpochStartDateTime.Reset() b.ExecutionOptimistic.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -180,14 +162,12 @@ func (b *beaconApiEthV1EventsBlockBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -196,7 +176,7 @@ func (b *beaconApiEthV1EventsBlockBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 31) + row := make(map[string]any, 28) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -207,7 +187,6 @@ func (b *beaconApiEthV1EventsBlockBatch) Snapshot() []map[string]any { row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["execution_optimistic"] = b.ExecutionOptimistic.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -240,14 +219,12 @@ func (b *beaconApiEthV1EventsBlockBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go index c2a71ea5..42189cc2 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_block_gossip.gen.go @@ -22,7 +22,6 @@ type beaconApiEthV1EventsBlockGossipBatch struct { Epoch proto.ColUInt32 EpochStartDateTime proto.ColDateTime MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -35,14 +34,12 @@ type beaconApiEthV1EventsBlockGossipBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -55,7 +52,6 @@ func newbeaconApiEthV1EventsBlockGossipBatch() *beaconApiEthV1EventsBlockGossipB MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -66,7 +62,6 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) Rows() int { func (b *beaconApiEthV1EventsBlockGossipBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -79,19 +74,16 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -104,7 +96,6 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -112,11 +103,6 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) appendMetadata(event *xatu.Decora b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsBlockGossipBatch) Input() proto.Input { @@ -130,7 +116,6 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) Input() proto.Input { {Name: "epoch", Data: &b.Epoch}, {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -143,14 +128,12 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -164,7 +147,6 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) Reset() { b.Epoch.Reset() b.EpochStartDateTime.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -177,14 +159,12 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -193,7 +173,7 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 30) + row := make(map[string]any, 27) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -203,7 +183,6 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) Snapshot() []map[string]any { row["epoch"] = b.Epoch.Row(i) row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -236,14 +215,12 @@ func (b *beaconApiEthV1EventsBlockGossipBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go index bf5cf27c..83d805ac 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_chain_reorg.gen.go @@ -27,7 +27,6 @@ type beaconApiEthV1EventsChainReorgBatch struct { EpochStartDateTime proto.ColDateTime ExecutionOptimistic proto.ColBool MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -40,14 +39,12 @@ type beaconApiEthV1EventsChainReorgBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -63,7 +60,6 @@ func newbeaconApiEthV1EventsChainReorgBatch() *beaconApiEthV1EventsChainReorgBat MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -74,7 +70,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) Rows() int { func (b *beaconApiEthV1EventsChainReorgBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -87,19 +82,16 @@ func (b *beaconApiEthV1EventsChainReorgBatch) appendMetadata(event *xatu.Decorat b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -112,7 +104,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) appendMetadata(event *xatu.Decorat b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -120,11 +111,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) appendMetadata(event *xatu.Decorat b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsChainReorgBatch) Input() proto.Input { @@ -143,7 +129,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) Input() proto.Input { {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "execution_optimistic", Data: &b.ExecutionOptimistic}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -156,14 +141,12 @@ func (b *beaconApiEthV1EventsChainReorgBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -182,7 +165,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) Reset() { b.EpochStartDateTime.Reset() b.ExecutionOptimistic.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -195,14 +177,12 @@ func (b *beaconApiEthV1EventsChainReorgBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -211,7 +191,7 @@ func (b *beaconApiEthV1EventsChainReorgBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 35) + row := make(map[string]any, 32) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -226,7 +206,6 @@ func (b *beaconApiEthV1EventsChainReorgBatch) Snapshot() []map[string]any { row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["execution_optimistic"] = b.ExecutionOptimistic.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -259,14 +238,12 @@ func (b *beaconApiEthV1EventsChainReorgBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go index 640f147d..0858380f 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_contribution_and_proof.gen.go @@ -28,7 +28,6 @@ type beaconApiEthV1EventsContributionAndProofBatch struct { SelectionProof proto.ColStr Signature proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -41,14 +40,12 @@ type beaconApiEthV1EventsContributionAndProofBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -61,7 +58,6 @@ func newbeaconApiEthV1EventsContributionAndProofBatch() *beaconApiEthV1EventsCon MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -72,7 +68,6 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) Rows() int { func (b *beaconApiEthV1EventsContributionAndProofBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -85,19 +80,16 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) appendMetadata(event *xa b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -110,7 +102,6 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) appendMetadata(event *xa b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -118,11 +109,6 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) appendMetadata(event *xa b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsContributionAndProofBatch) Input() proto.Input { @@ -142,7 +128,6 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) Input() proto.Input { {Name: "selection_proof", Data: &b.SelectionProof}, {Name: "signature", Data: &b.Signature}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -155,14 +140,12 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -182,7 +165,6 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) Reset() { b.SelectionProof.Reset() b.Signature.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -195,14 +177,12 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -211,7 +191,7 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) Snapshot() []map[string] out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 36) + row := make(map[string]any, 33) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["aggregator_index"] = b.AggregatorIndex.Row(i) @@ -227,7 +207,6 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) Snapshot() []map[string] row["selection_proof"] = b.SelectionProof.Row(i) row["signature"] = b.Signature.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -260,14 +239,12 @@ func (b *beaconApiEthV1EventsContributionAndProofBatch) Snapshot() []map[string] } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go index e70b399a..ed72fef2 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go @@ -23,8 +23,8 @@ type beaconApiEthV1EventsDataColumnSidecarBatch struct { BlockRoot route.SafeColFixedStr ColumnIndex proto.ColUInt64 KzgCommitmentsCount proto.ColUInt32 + KzgCommitments *proto.ColArr[[]byte] MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -37,27 +37,29 @@ type beaconApiEthV1EventsDataColumnSidecarBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } func newbeaconApiEthV1EventsDataColumnSidecarBatch() *beaconApiEthV1EventsDataColumnSidecarBatch { return &beaconApiEthV1EventsDataColumnSidecarBatch{ - EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + KzgCommitments: func() *proto.ColArr[[]byte] { + var fs route.SafeColFixedStr + fs.SetSize(98) + return proto.NewArray[[]byte](&fs) + }(), MetaClientIP: new(proto.ColIPv6).Nullable(), MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -68,7 +70,6 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Rows() int { func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -81,19 +82,16 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendMetadata(event *xatu. b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -106,7 +104,6 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendMetadata(event *xatu. b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -114,11 +111,6 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendMetadata(event *xatu. b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Input() proto.Input { @@ -133,8 +125,8 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Input() proto.Input { {Name: "block_root", Data: &b.BlockRoot}, {Name: "column_index", Data: &b.ColumnIndex}, {Name: "kzg_commitments_count", Data: &b.KzgCommitmentsCount}, + {Name: "kzg_commitments", Data: b.KzgCommitments}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -147,14 +139,12 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -169,8 +159,8 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Reset() { b.BlockRoot.Reset() b.ColumnIndex.Reset() b.KzgCommitmentsCount.Reset() + b.KzgCommitments.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -183,14 +173,12 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -199,7 +187,7 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Snapshot() []map[string]any out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 32) + row := make(map[string]any, 30) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -210,8 +198,8 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Snapshot() []map[string]any row["block_root"] = string(b.BlockRoot.Row(i)) row["column_index"] = b.ColumnIndex.Row(i) row["kzg_commitments_count"] = b.KzgCommitmentsCount.Row(i) + row["kzg_commitments"] = route.ByteSlicesToStrings(b.KzgCommitments.Row(i)) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -244,14 +232,12 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Snapshot() []map[string]any } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go index 3fd7cf00..f60b6115 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go @@ -104,6 +104,14 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendPayload(event *xatu.D } else { b.KzgCommitmentsCount.Append(0) } + + commitments := sidecar.GetKzgCommitments() //nolint:staticcheck // deprecated but still populated + byteSlices := make([][]byte, len(commitments)) + for i, c := range commitments { + byteSlices[i] = []byte(c) + } + + b.KzgCommitments.Append(byteSlices) } func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendAdditionalData( diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go index b40e5adc..f1d40c86 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_finalized_checkpoint.gen.go @@ -21,7 +21,6 @@ type beaconApiEthV1EventsFinalizedCheckpointBatch struct { EpochStartDateTime proto.ColDateTime ExecutionOptimistic proto.ColBool MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -34,14 +33,12 @@ type beaconApiEthV1EventsFinalizedCheckpointBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -55,7 +52,6 @@ func newbeaconApiEthV1EventsFinalizedCheckpointBatch() *beaconApiEthV1EventsFina MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -66,7 +62,6 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Rows() int { func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -79,19 +74,16 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) appendMetadata(event *xat b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -104,7 +96,6 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) appendMetadata(event *xat b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -112,11 +103,6 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) appendMetadata(event *xat b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Input() proto.Input { @@ -129,7 +115,6 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Input() proto.Input { {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "execution_optimistic", Data: &b.ExecutionOptimistic}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -142,14 +127,12 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -162,7 +145,6 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Reset() { b.EpochStartDateTime.Reset() b.ExecutionOptimistic.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -175,14 +157,12 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -191,7 +171,7 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Snapshot() []map[string]a out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 29) + row := make(map[string]any, 26) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["block"] = string(b.Block.Row(i)) @@ -200,7 +180,6 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Snapshot() []map[string]a row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["execution_optimistic"] = b.ExecutionOptimistic.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -233,14 +212,12 @@ func (b *beaconApiEthV1EventsFinalizedCheckpointBatch) Snapshot() []map[string]a } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go index 03901ab4..b84422e6 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_head.gen.go @@ -26,7 +26,6 @@ type beaconApiEthV1EventsHeadBatch struct { PreviousDutyDependentRoot route.SafeColFixedStr CurrentDutyDependentRoot route.SafeColFixedStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -39,14 +38,12 @@ type beaconApiEthV1EventsHeadBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -61,7 +58,6 @@ func newbeaconApiEthV1EventsHeadBatch() *beaconApiEthV1EventsHeadBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -72,7 +68,6 @@ func (b *beaconApiEthV1EventsHeadBatch) Rows() int { func (b *beaconApiEthV1EventsHeadBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -85,19 +80,16 @@ func (b *beaconApiEthV1EventsHeadBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -110,7 +102,6 @@ func (b *beaconApiEthV1EventsHeadBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -118,11 +109,6 @@ func (b *beaconApiEthV1EventsHeadBatch) appendMetadata(event *xatu.DecoratedEven b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsHeadBatch) Input() proto.Input { @@ -140,7 +126,6 @@ func (b *beaconApiEthV1EventsHeadBatch) Input() proto.Input { {Name: "previous_duty_dependent_root", Data: &b.PreviousDutyDependentRoot}, {Name: "current_duty_dependent_root", Data: &b.CurrentDutyDependentRoot}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -153,14 +138,12 @@ func (b *beaconApiEthV1EventsHeadBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -178,7 +161,6 @@ func (b *beaconApiEthV1EventsHeadBatch) Reset() { b.PreviousDutyDependentRoot.Reset() b.CurrentDutyDependentRoot.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -191,14 +173,12 @@ func (b *beaconApiEthV1EventsHeadBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -207,7 +187,7 @@ func (b *beaconApiEthV1EventsHeadBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 34) + row := make(map[string]any, 31) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -221,7 +201,6 @@ func (b *beaconApiEthV1EventsHeadBatch) Snapshot() []map[string]any { row["previous_duty_dependent_root"] = string(b.PreviousDutyDependentRoot.Row(i)) row["current_duty_dependent_root"] = string(b.CurrentDutyDependentRoot.Row(i)) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -254,14 +233,12 @@ func (b *beaconApiEthV1EventsHeadBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go index 0dbfd352..735f9ed9 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_voluntary_exit.gen.go @@ -24,7 +24,6 @@ type beaconApiEthV1EventsVoluntaryExitBatch struct { ValidatorIndex proto.ColUInt32 Signature proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -37,14 +36,12 @@ type beaconApiEthV1EventsVoluntaryExitBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -56,7 +53,6 @@ func newbeaconApiEthV1EventsVoluntaryExitBatch() *beaconApiEthV1EventsVoluntaryE MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -67,7 +63,6 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) Rows() int { func (b *beaconApiEthV1EventsVoluntaryExitBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -80,19 +75,16 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) appendMetadata(event *xatu.Deco b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -105,7 +97,6 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) appendMetadata(event *xatu.Deco b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -113,11 +104,6 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) appendMetadata(event *xatu.Deco b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1EventsVoluntaryExitBatch) Input() proto.Input { @@ -133,7 +119,6 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) Input() proto.Input { {Name: "validator_index", Data: &b.ValidatorIndex}, {Name: "signature", Data: &b.Signature}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -146,14 +131,12 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -169,7 +152,6 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) Reset() { b.ValidatorIndex.Reset() b.Signature.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -182,14 +164,12 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -198,7 +178,7 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 32) + row := make(map[string]any, 29) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["epoch"] = b.Epoch.Row(i) @@ -210,7 +190,6 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) Snapshot() []map[string]any { row["validator_index"] = b.ValidatorIndex.Row(i) row["signature"] = b.Signature.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -243,14 +222,12 @@ func (b *beaconApiEthV1EventsVoluntaryExitBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go index ba8e9026..c247177d 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_proposer_duty.gen.go @@ -22,7 +22,6 @@ type beaconApiEthV1ProposerDutyBatch struct { ProposerValidatorIndex proto.ColUInt32 ProposerPubkey proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -35,14 +34,12 @@ type beaconApiEthV1ProposerDutyBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -54,7 +51,6 @@ func newbeaconApiEthV1ProposerDutyBatch() *beaconApiEthV1ProposerDutyBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -65,7 +61,6 @@ func (b *beaconApiEthV1ProposerDutyBatch) Rows() int { func (b *beaconApiEthV1ProposerDutyBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -78,19 +73,16 @@ func (b *beaconApiEthV1ProposerDutyBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -103,7 +95,6 @@ func (b *beaconApiEthV1ProposerDutyBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -111,11 +102,6 @@ func (b *beaconApiEthV1ProposerDutyBatch) appendMetadata(event *xatu.DecoratedEv b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1ProposerDutyBatch) Input() proto.Input { @@ -129,7 +115,6 @@ func (b *beaconApiEthV1ProposerDutyBatch) Input() proto.Input { {Name: "proposer_validator_index", Data: &b.ProposerValidatorIndex}, {Name: "proposer_pubkey", Data: &b.ProposerPubkey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -142,14 +127,12 @@ func (b *beaconApiEthV1ProposerDutyBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -163,7 +146,6 @@ func (b *beaconApiEthV1ProposerDutyBatch) Reset() { b.ProposerValidatorIndex.Reset() b.ProposerPubkey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -176,14 +158,12 @@ func (b *beaconApiEthV1ProposerDutyBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -192,7 +172,7 @@ func (b *beaconApiEthV1ProposerDutyBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 30) + row := make(map[string]any, 27) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -202,7 +182,6 @@ func (b *beaconApiEthV1ProposerDutyBatch) Snapshot() []map[string]any { row["proposer_validator_index"] = b.ProposerValidatorIndex.Row(i) row["proposer_pubkey"] = b.ProposerPubkey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -235,14 +214,12 @@ func (b *beaconApiEthV1ProposerDutyBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go index e69e44bf..449109f0 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_validator_attestation_data.gen.go @@ -31,7 +31,6 @@ type beaconApiEthV1ValidatorAttestationDataBatch struct { RequestDuration proto.ColUInt32 RequestSlotStartDiff proto.ColUInt32 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -44,14 +43,12 @@ type beaconApiEthV1ValidatorAttestationDataBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -66,7 +63,6 @@ func newbeaconApiEthV1ValidatorAttestationDataBatch() *beaconApiEthV1ValidatorAt MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -77,7 +73,6 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) Rows() int { func (b *beaconApiEthV1ValidatorAttestationDataBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -90,19 +85,16 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) appendMetadata(event *xatu b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -115,7 +107,6 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) appendMetadata(event *xatu b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -123,11 +114,6 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) appendMetadata(event *xatu b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV1ValidatorAttestationDataBatch) Input() proto.Input { @@ -150,7 +136,6 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) Input() proto.Input { {Name: "request_duration", Data: &b.RequestDuration}, {Name: "request_slot_start_diff", Data: &b.RequestSlotStartDiff}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -163,14 +148,12 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -193,7 +176,6 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) Reset() { b.RequestDuration.Reset() b.RequestSlotStartDiff.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -206,14 +188,12 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -222,7 +202,7 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) Snapshot() []map[string]an out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 39) + row := make(map[string]any, 36) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -241,7 +221,6 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) Snapshot() []map[string]an row["request_duration"] = b.RequestDuration.Row(i) row["request_slot_start_diff"] = b.RequestSlotStartDiff.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -274,14 +253,12 @@ func (b *beaconApiEthV1ValidatorAttestationDataBatch) Snapshot() []map[string]an } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go index 09bc3459..7778fdf5 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.gen.go @@ -28,9 +28,9 @@ type beaconApiEthV2BeaconBlockBatch struct { ProposerIndex proto.ColUInt32 Eth1DataBlockHash route.SafeColFixedStr Eth1DataDepositRoot route.SafeColFixedStr - ExecutionPayloadBlockHash route.SafeColFixedStr - ExecutionPayloadBlockNumber proto.ColUInt32 - ExecutionPayloadFeeRecipient proto.ColStr + ExecutionPayloadBlockHash *proto.ColNullable[[]byte] + ExecutionPayloadBlockNumber *proto.ColNullable[uint32] + ExecutionPayloadFeeRecipient *proto.ColNullable[string] ExecutionPayloadBaseFeePerGas *proto.ColNullable[proto.UInt128] ExecutionPayloadBlobGasUsed *proto.ColNullable[uint64] ExecutionPayloadExcessBlobGas *proto.ColNullable[uint64] @@ -42,7 +42,6 @@ type beaconApiEthV2BeaconBlockBatch struct { ExecutionPayloadTransactionsTotalBytes *proto.ColNullable[uint32] ExecutionPayloadTransactionsTotalBytesCompressed *proto.ColNullable[uint32] MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -55,14 +54,12 @@ type beaconApiEthV2BeaconBlockBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -76,7 +73,9 @@ func newbeaconApiEthV2BeaconBlockBatch() *beaconApiEthV2BeaconBlockBatch { StateRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), Eth1DataBlockHash: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), Eth1DataDepositRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - ExecutionPayloadBlockHash: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + ExecutionPayloadBlockHash: route.NewNullableFixedStr(66), + ExecutionPayloadBlockNumber: new(proto.ColUInt32).Nullable(), + ExecutionPayloadFeeRecipient: new(proto.ColStr).Nullable(), ExecutionPayloadBaseFeePerGas: new(proto.ColUInt128).Nullable(), ExecutionPayloadBlobGasUsed: new(proto.ColUInt64).Nullable(), ExecutionPayloadExcessBlobGas: new(proto.ColUInt64).Nullable(), @@ -92,7 +91,6 @@ func newbeaconApiEthV2BeaconBlockBatch() *beaconApiEthV2BeaconBlockBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -103,7 +101,6 @@ func (b *beaconApiEthV2BeaconBlockBatch) Rows() int { func (b *beaconApiEthV2BeaconBlockBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -116,19 +113,16 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -141,7 +135,6 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -149,11 +142,6 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendMetadata(event *xatu.DecoratedEve b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV2BeaconBlockBatch) Input() proto.Input { @@ -173,9 +161,9 @@ func (b *beaconApiEthV2BeaconBlockBatch) Input() proto.Input { {Name: "proposer_index", Data: &b.ProposerIndex}, {Name: "eth1_data_block_hash", Data: &b.Eth1DataBlockHash}, {Name: "eth1_data_deposit_root", Data: &b.Eth1DataDepositRoot}, - {Name: "execution_payload_block_hash", Data: &b.ExecutionPayloadBlockHash}, - {Name: "execution_payload_block_number", Data: &b.ExecutionPayloadBlockNumber}, - {Name: "execution_payload_fee_recipient", Data: &b.ExecutionPayloadFeeRecipient}, + {Name: "execution_payload_block_hash", Data: b.ExecutionPayloadBlockHash}, + {Name: "execution_payload_block_number", Data: b.ExecutionPayloadBlockNumber}, + {Name: "execution_payload_fee_recipient", Data: b.ExecutionPayloadFeeRecipient}, {Name: "execution_payload_base_fee_per_gas", Data: b.ExecutionPayloadBaseFeePerGas}, {Name: "execution_payload_blob_gas_used", Data: b.ExecutionPayloadBlobGasUsed}, {Name: "execution_payload_excess_blob_gas", Data: b.ExecutionPayloadExcessBlobGas}, @@ -187,7 +175,6 @@ func (b *beaconApiEthV2BeaconBlockBatch) Input() proto.Input { {Name: "execution_payload_transactions_total_bytes", Data: b.ExecutionPayloadTransactionsTotalBytes}, {Name: "execution_payload_transactions_total_bytes_compressed", Data: b.ExecutionPayloadTransactionsTotalBytesCompressed}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -200,14 +187,12 @@ func (b *beaconApiEthV2BeaconBlockBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -241,7 +226,6 @@ func (b *beaconApiEthV2BeaconBlockBatch) Reset() { b.ExecutionPayloadTransactionsTotalBytes.Reset() b.ExecutionPayloadTransactionsTotalBytesCompressed.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -254,14 +238,12 @@ func (b *beaconApiEthV2BeaconBlockBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -270,7 +252,7 @@ func (b *beaconApiEthV2BeaconBlockBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 50) + row := make(map[string]any, 47) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -294,9 +276,21 @@ func (b *beaconApiEthV2BeaconBlockBatch) Snapshot() []map[string]any { row["proposer_index"] = b.ProposerIndex.Row(i) row["eth1_data_block_hash"] = string(b.Eth1DataBlockHash.Row(i)) row["eth1_data_deposit_root"] = string(b.Eth1DataDepositRoot.Row(i)) - row["execution_payload_block_hash"] = string(b.ExecutionPayloadBlockHash.Row(i)) - row["execution_payload_block_number"] = b.ExecutionPayloadBlockNumber.Row(i) - row["execution_payload_fee_recipient"] = b.ExecutionPayloadFeeRecipient.Row(i) + if v := b.ExecutionPayloadBlockHash.Row(i); v.Set { + row["execution_payload_block_hash"] = string(v.Value) + } else { + row["execution_payload_block_hash"] = nil + } + if v := b.ExecutionPayloadBlockNumber.Row(i); v.Set { + row["execution_payload_block_number"] = v.Value + } else { + row["execution_payload_block_number"] = nil + } + if v := b.ExecutionPayloadFeeRecipient.Row(i); v.Set { + row["execution_payload_fee_recipient"] = v.Value + } else { + row["execution_payload_fee_recipient"] = nil + } if v := b.ExecutionPayloadBaseFeePerGas.Row(i); v.Set { row["execution_payload_base_fee_per_gas"] = route.UInt128ToString(v.Value) } else { @@ -340,7 +334,6 @@ func (b *beaconApiEthV2BeaconBlockBatch) Snapshot() []map[string]any { row["execution_payload_transactions_total_bytes_compressed"] = nil } row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -373,14 +366,12 @@ func (b *beaconApiEthV2BeaconBlockBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.go b/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.go index 525fe149..0ebe682c 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v2_beacon_block.go @@ -273,9 +273,9 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendEth1Data(eth1Data *ethv1.Eth1Data } func (b *beaconApiEthV2BeaconBlockBatch) appendNoExecutionPayload() { - b.ExecutionPayloadBlockHash.Append(nil) - b.ExecutionPayloadBlockNumber.Append(0) - b.ExecutionPayloadFeeRecipient.Append("") + b.ExecutionPayloadBlockHash.Append(proto.Nullable[[]byte]{}) + b.ExecutionPayloadBlockNumber.Append(proto.Nullable[uint32]{}) + b.ExecutionPayloadFeeRecipient.Append(proto.Nullable[string]{}) b.ExecutionPayloadBaseFeePerGas.Append(proto.Nullable[proto.UInt128]{}) b.ExecutionPayloadBlobGasUsed.Append(proto.Nullable[uint64]{}) b.ExecutionPayloadExcessBlobGas.Append(proto.Nullable[uint64]{}) @@ -301,15 +301,15 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendExecutionPayloadV2( b.ExecutionPayloadBaseFeePerGas.Append(proto.NewNullable[proto.UInt128](baseFeePerGas)) } - b.ExecutionPayloadBlockHash.Append([]byte(payload.GetBlockHash())) - b.ExecutionPayloadFeeRecipient.Append(payload.GetFeeRecipient()) + b.ExecutionPayloadBlockHash.Append(proto.NewNullable[[]byte]([]byte(payload.GetBlockHash()))) + b.ExecutionPayloadFeeRecipient.Append(proto.NewNullable[string](payload.GetFeeRecipient())) b.ExecutionPayloadStateRoot.Append([]byte(payload.GetStateRoot())) b.ExecutionPayloadParentHash.Append([]byte(payload.GetParentHash())) if blockNumber := payload.GetBlockNumber(); blockNumber != nil { - b.ExecutionPayloadBlockNumber.Append(uint32(blockNumber.GetValue())) //nolint:gosec // block number fits uint32 + b.ExecutionPayloadBlockNumber.Append(proto.NewNullable[uint32](uint32(blockNumber.GetValue()))) //nolint:gosec // block number fits uint32 } else { - b.ExecutionPayloadBlockNumber.Append(0) + b.ExecutionPayloadBlockNumber.Append(proto.Nullable[uint32]{}) } if gasLimit := payload.GetGasLimit(); gasLimit != nil { @@ -346,15 +346,15 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendExecutionPayloadCapellaV2( b.ExecutionPayloadBaseFeePerGas.Append(proto.NewNullable[proto.UInt128](baseFeePerGas)) } - b.ExecutionPayloadBlockHash.Append([]byte(payload.GetBlockHash())) - b.ExecutionPayloadFeeRecipient.Append(payload.GetFeeRecipient()) + b.ExecutionPayloadBlockHash.Append(proto.NewNullable[[]byte]([]byte(payload.GetBlockHash()))) + b.ExecutionPayloadFeeRecipient.Append(proto.NewNullable[string](payload.GetFeeRecipient())) b.ExecutionPayloadStateRoot.Append([]byte(payload.GetStateRoot())) b.ExecutionPayloadParentHash.Append([]byte(payload.GetParentHash())) if blockNumber := payload.GetBlockNumber(); blockNumber != nil { - b.ExecutionPayloadBlockNumber.Append(uint32(blockNumber.GetValue())) //nolint:gosec // block number fits uint32 + b.ExecutionPayloadBlockNumber.Append(proto.NewNullable[uint32](uint32(blockNumber.GetValue()))) //nolint:gosec // block number fits uint32 } else { - b.ExecutionPayloadBlockNumber.Append(0) + b.ExecutionPayloadBlockNumber.Append(proto.Nullable[uint32]{}) } if gasLimit := payload.GetGasLimit(); gasLimit != nil { @@ -391,15 +391,15 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendExecutionPayloadDeneb( b.ExecutionPayloadBaseFeePerGas.Append(proto.NewNullable[proto.UInt128](baseFeePerGas)) } - b.ExecutionPayloadBlockHash.Append([]byte(payload.GetBlockHash())) - b.ExecutionPayloadFeeRecipient.Append(payload.GetFeeRecipient()) + b.ExecutionPayloadBlockHash.Append(proto.NewNullable[[]byte]([]byte(payload.GetBlockHash()))) + b.ExecutionPayloadFeeRecipient.Append(proto.NewNullable[string](payload.GetFeeRecipient())) b.ExecutionPayloadStateRoot.Append([]byte(payload.GetStateRoot())) b.ExecutionPayloadParentHash.Append([]byte(payload.GetParentHash())) if blockNumber := payload.GetBlockNumber(); blockNumber != nil { - b.ExecutionPayloadBlockNumber.Append(uint32(blockNumber.GetValue())) //nolint:gosec // block number fits uint32 + b.ExecutionPayloadBlockNumber.Append(proto.NewNullable[uint32](uint32(blockNumber.GetValue()))) //nolint:gosec // block number fits uint32 } else { - b.ExecutionPayloadBlockNumber.Append(0) + b.ExecutionPayloadBlockNumber.Append(proto.Nullable[uint32]{}) } if blobGasUsed := payload.GetBlobGasUsed(); blobGasUsed != nil { @@ -445,15 +445,15 @@ func (b *beaconApiEthV2BeaconBlockBatch) appendExecutionPayloadElectra( b.ExecutionPayloadBaseFeePerGas.Append(proto.NewNullable[proto.UInt128](baseFeePerGas)) } - b.ExecutionPayloadBlockHash.Append([]byte(payload.GetBlockHash())) - b.ExecutionPayloadFeeRecipient.Append(payload.GetFeeRecipient()) + b.ExecutionPayloadBlockHash.Append(proto.NewNullable[[]byte]([]byte(payload.GetBlockHash()))) + b.ExecutionPayloadFeeRecipient.Append(proto.NewNullable[string](payload.GetFeeRecipient())) b.ExecutionPayloadStateRoot.Append([]byte(payload.GetStateRoot())) b.ExecutionPayloadParentHash.Append([]byte(payload.GetParentHash())) if blockNumber := payload.GetBlockNumber(); blockNumber != nil { - b.ExecutionPayloadBlockNumber.Append(uint32(blockNumber.GetValue())) //nolint:gosec // block number fits uint32 + b.ExecutionPayloadBlockNumber.Append(proto.NewNullable[uint32](uint32(blockNumber.GetValue()))) //nolint:gosec // block number fits uint32 } else { - b.ExecutionPayloadBlockNumber.Append(0) + b.ExecutionPayloadBlockNumber.Append(proto.Nullable[uint32]{}) } if blobGasUsed := payload.GetBlobGasUsed(); blobGasUsed != nil { diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go index eea9f1c3..affe8a09 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v3_validator_block.gen.go @@ -34,7 +34,6 @@ type beaconApiEthV3ValidatorBlockBatch struct { ExecutionPayloadTransactionsTotalBytes *proto.ColNullable[uint32] ExecutionPayloadTransactionsTotalBytesCompressed *proto.ColNullable[uint32] MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -47,14 +46,12 @@ type beaconApiEthV3ValidatorBlockBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaConsensusVersion proto.ColStr MetaConsensusVersionMajor proto.ColStr MetaConsensusVersionMinor proto.ColStr MetaConsensusVersionPatch proto.ColStr MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -78,7 +75,6 @@ func newbeaconApiEthV3ValidatorBlockBatch() *beaconApiEthV3ValidatorBlockBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -89,7 +85,6 @@ func (b *beaconApiEthV3ValidatorBlockBatch) Rows() int { func (b *beaconApiEthV3ValidatorBlockBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -102,19 +97,16 @@ func (b *beaconApiEthV3ValidatorBlockBatch) appendMetadata(event *xatu.Decorated b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaConsensusVersion.Append("") b.MetaConsensusVersionMajor.Append("") b.MetaConsensusVersionMinor.Append("") b.MetaConsensusVersionPatch.Append("") b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -127,7 +119,6 @@ func (b *beaconApiEthV3ValidatorBlockBatch) appendMetadata(event *xatu.Decorated b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) b.MetaConsensusVersion.Append(cvNorm) @@ -135,11 +126,6 @@ func (b *beaconApiEthV3ValidatorBlockBatch) appendMetadata(event *xatu.Decorated b.MetaConsensusVersionMinor.Append(cvMinor) b.MetaConsensusVersionPatch.Append(cvPatch) b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *beaconApiEthV3ValidatorBlockBatch) Input() proto.Input { @@ -165,7 +151,6 @@ func (b *beaconApiEthV3ValidatorBlockBatch) Input() proto.Input { {Name: "execution_payload_transactions_total_bytes", Data: b.ExecutionPayloadTransactionsTotalBytes}, {Name: "execution_payload_transactions_total_bytes_compressed", Data: b.ExecutionPayloadTransactionsTotalBytesCompressed}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -178,14 +163,12 @@ func (b *beaconApiEthV3ValidatorBlockBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -211,7 +194,6 @@ func (b *beaconApiEthV3ValidatorBlockBatch) Reset() { b.ExecutionPayloadTransactionsTotalBytes.Reset() b.ExecutionPayloadTransactionsTotalBytesCompressed.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -224,14 +206,12 @@ func (b *beaconApiEthV3ValidatorBlockBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaConsensusVersion.Reset() b.MetaConsensusVersionMajor.Reset() b.MetaConsensusVersionMinor.Reset() b.MetaConsensusVersionPatch.Reset() b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -240,7 +220,7 @@ func (b *beaconApiEthV3ValidatorBlockBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 42) + row := make(map[string]any, 39) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -310,7 +290,6 @@ func (b *beaconApiEthV3ValidatorBlockBatch) Snapshot() []map[string]any { row["execution_payload_transactions_total_bytes_compressed"] = nil } row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -343,14 +322,12 @@ func (b *beaconApiEthV3ValidatorBlockBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go index dd471da7..01bb5e16 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_blob_sidecar.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,59 +11,32 @@ import ( const canonicalBeaconBlobSidecarTableName route.TableName = "canonical_beacon_blob_sidecar" type canonicalBeaconBlobSidecarBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockParentRoot route.SafeColFixedStr - VersionedHash route.SafeColFixedStr - KzgCommitment route.SafeColFixedStr - KzgProof route.SafeColFixedStr - ProposerIndex proto.ColUInt32 - BlobIndex proto.ColUInt64 - BlobSize proto.ColUInt32 - BlobEmptySize *proto.ColNullable[uint32] - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockParentRoot route.SafeColFixedStr + VersionedHash route.SafeColFixedStr + KzgCommitment route.SafeColFixedStr + KzgProof route.SafeColFixedStr + ProposerIndex proto.ColUInt32 + BlobIndex proto.ColUInt64 + BlobSize proto.ColUInt32 + BlobEmptySize *proto.ColNullable[uint32] + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlobSidecarBatch() *canonicalBeaconBlobSidecarBatch { return &canonicalBeaconBlobSidecarBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - BlockParentRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - VersionedHash: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - KzgCommitment: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(98); return c }(), - KzgProof: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(98); return c }(), - BlobEmptySize: new(proto.ColUInt32).Nullable(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + BlockParentRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + VersionedHash: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + KzgCommitment: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(98); return c }(), + KzgProof: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(98); return c }(), + BlobEmptySize: new(proto.ColUInt32).Nullable(), } } @@ -75,58 +46,11 @@ func (b *canonicalBeaconBlobSidecarBatch) Rows() int { func (b *canonicalBeaconBlobSidecarBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlobSidecarBatch) Input() proto.Input { @@ -145,28 +69,7 @@ func (b *canonicalBeaconBlobSidecarBatch) Input() proto.Input { {Name: "blob_index", Data: &b.BlobIndex}, {Name: "blob_size", Data: &b.BlobSize}, {Name: "blob_empty_size", Data: b.BlobEmptySize}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -185,28 +88,7 @@ func (b *canonicalBeaconBlobSidecarBatch) Reset() { b.BlobIndex.Reset() b.BlobSize.Reset() b.BlobEmptySize.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -215,7 +97,7 @@ func (b *canonicalBeaconBlobSidecarBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 36) + row := make(map[string]any, 15) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -234,48 +116,7 @@ func (b *canonicalBeaconBlobSidecarBatch) Snapshot() []map[string]any { } else { row["blob_empty_size"] = nil } - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go index d145f5a5..5e383684 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -40,28 +38,7 @@ type canonicalBeaconBlockBatch struct { ExecutionPayloadTransactionsCount *proto.ColNullable[uint32] ExecutionPayloadTransactionsTotalBytes *proto.ColNullable[uint32] ExecutionPayloadTransactionsTotalBytesCompressed *proto.ColNullable[uint32] - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -87,12 +64,6 @@ func newcanonicalBeaconBlockBatch() *canonicalBeaconBlockBatch { ExecutionPayloadTransactionsCount: new(proto.ColUInt32).Nullable(), ExecutionPayloadTransactionsTotalBytes: new(proto.ColUInt32).Nullable(), ExecutionPayloadTransactionsTotalBytesCompressed: new(proto.ColUInt32).Nullable(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -102,58 +73,11 @@ func (b *canonicalBeaconBlockBatch) Rows() int { func (b *canonicalBeaconBlockBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockBatch) Input() proto.Input { @@ -185,28 +109,7 @@ func (b *canonicalBeaconBlockBatch) Input() proto.Input { {Name: "execution_payload_transactions_count", Data: b.ExecutionPayloadTransactionsCount}, {Name: "execution_payload_transactions_total_bytes", Data: b.ExecutionPayloadTransactionsTotalBytes}, {Name: "execution_payload_transactions_total_bytes_compressed", Data: b.ExecutionPayloadTransactionsTotalBytesCompressed}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -238,28 +141,7 @@ func (b *canonicalBeaconBlockBatch) Reset() { b.ExecutionPayloadTransactionsCount.Reset() b.ExecutionPayloadTransactionsTotalBytes.Reset() b.ExecutionPayloadTransactionsTotalBytesCompressed.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -268,7 +150,7 @@ func (b *canonicalBeaconBlockBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 49) + row := make(map[string]any, 28) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -356,48 +238,7 @@ func (b *canonicalBeaconBlockBatch) Snapshot() []map[string]any { } else { row["execution_payload_transactions_total_bytes_compressed"] = nil } - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go index 81e2ef4d..24b3fa16 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,73 +11,46 @@ import ( const canonicalBeaconBlockAttesterSlashingTableName route.TableName = "canonical_beacon_block_attester_slashing" type canonicalBeaconBlockAttesterSlashingBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockVersion proto.ColStr - Attestation1AttestingIndices *proto.ColArr[uint32] - Attestation1Signature proto.ColStr - Attestation1DataBeaconBlockRoot route.SafeColFixedStr - Attestation1DataSlot proto.ColUInt32 - Attestation1DataIndex proto.ColUInt32 - Attestation1DataSourceEpoch proto.ColUInt32 - Attestation1DataSourceRoot route.SafeColFixedStr - Attestation1DataTargetEpoch proto.ColUInt32 - Attestation1DataTargetRoot route.SafeColFixedStr - Attestation2AttestingIndices *proto.ColArr[uint32] - Attestation2Signature proto.ColStr - Attestation2DataBeaconBlockRoot route.SafeColFixedStr - Attestation2DataSlot proto.ColUInt32 - Attestation2DataIndex proto.ColUInt32 - Attestation2DataSourceEpoch proto.ColUInt32 - Attestation2DataSourceRoot route.SafeColFixedStr - Attestation2DataTargetEpoch proto.ColUInt32 - Attestation2DataTargetRoot route.SafeColFixedStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockVersion proto.ColStr + Attestation1AttestingIndices *proto.ColArr[uint32] + Attestation1Signature proto.ColStr + Attestation1DataBeaconBlockRoot route.SafeColFixedStr + Attestation1DataSlot proto.ColUInt32 + Attestation1DataIndex proto.ColUInt32 + Attestation1DataSourceEpoch proto.ColUInt32 + Attestation1DataSourceRoot route.SafeColFixedStr + Attestation1DataTargetEpoch proto.ColUInt32 + Attestation1DataTargetRoot route.SafeColFixedStr + Attestation2AttestingIndices *proto.ColArr[uint32] + Attestation2Signature proto.ColStr + Attestation2DataBeaconBlockRoot route.SafeColFixedStr + Attestation2DataSlot proto.ColUInt32 + Attestation2DataIndex proto.ColUInt32 + Attestation2DataSourceEpoch proto.ColUInt32 + Attestation2DataSourceRoot route.SafeColFixedStr + Attestation2DataTargetEpoch proto.ColUInt32 + Attestation2DataTargetRoot route.SafeColFixedStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlockAttesterSlashingBatch() *canonicalBeaconBlockAttesterSlashingBatch { return &canonicalBeaconBlockAttesterSlashingBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - Attestation1AttestingIndices: proto.NewArray[uint32](new(proto.ColUInt32)), - Attestation1DataBeaconBlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - Attestation1DataSourceRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - Attestation1DataTargetRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - Attestation2AttestingIndices: proto.NewArray[uint32](new(proto.ColUInt32)), - Attestation2DataBeaconBlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - Attestation2DataSourceRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - Attestation2DataTargetRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + Attestation1AttestingIndices: proto.NewArray[uint32](new(proto.ColUInt32)), + Attestation1DataBeaconBlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + Attestation1DataSourceRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + Attestation1DataTargetRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + Attestation2AttestingIndices: proto.NewArray[uint32](new(proto.ColUInt32)), + Attestation2DataBeaconBlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + Attestation2DataSourceRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + Attestation2DataTargetRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), } } @@ -89,58 +60,11 @@ func (b *canonicalBeaconBlockAttesterSlashingBatch) Rows() int { func (b *canonicalBeaconBlockAttesterSlashingBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockAttesterSlashingBatch) Input() proto.Input { @@ -170,28 +94,7 @@ func (b *canonicalBeaconBlockAttesterSlashingBatch) Input() proto.Input { {Name: "attestation_2_data_source_root", Data: &b.Attestation2DataSourceRoot}, {Name: "attestation_2_data_target_epoch", Data: &b.Attestation2DataTargetEpoch}, {Name: "attestation_2_data_target_root", Data: &b.Attestation2DataTargetRoot}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -221,28 +124,7 @@ func (b *canonicalBeaconBlockAttesterSlashingBatch) Reset() { b.Attestation2DataSourceRoot.Reset() b.Attestation2DataTargetEpoch.Reset() b.Attestation2DataTargetRoot.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -251,7 +133,7 @@ func (b *canonicalBeaconBlockAttesterSlashingBatch) Snapshot() []map[string]any out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 47) + row := make(map[string]any, 26) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -277,48 +159,7 @@ func (b *canonicalBeaconBlockAttesterSlashingBatch) Snapshot() []map[string]any row["attestation_2_data_source_root"] = string(b.Attestation2DataSourceRoot.Row(i)) row["attestation_2_data_target_epoch"] = b.Attestation2DataTargetEpoch.Row(i) row["attestation_2_data_target_root"] = string(b.Attestation2DataTargetRoot.Row(i)) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing_test.go b/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing_test.go index a809fa9a..42f4c2f7 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_attester_slashing_test.go @@ -28,6 +28,6 @@ func TestSnapshot_canonical_beacon_block_attester_slashing(t *testing.T) { EthV2BeaconBlockAttesterSlashing: ðv1.AttesterSlashingV2{}, }, }, 1, map[string]any{ - "meta_client_name": "test-client", + "meta_network_name": "mainnet", }) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go index cad963a7..5983c847 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,52 +11,25 @@ import ( const canonicalBeaconBlockBlsToExecutionChangeTableName route.TableName = "canonical_beacon_block_bls_to_execution_change" type canonicalBeaconBlockBlsToExecutionChangeBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockVersion proto.ColStr - ExchangingMessageValidatorIndex proto.ColUInt32 - ExchangingMessageFromBlsPubkey proto.ColStr - ExchangingMessageToExecutionAddress route.SafeColFixedStr - ExchangingSignature proto.ColStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockVersion proto.ColStr + ExchangingMessageValidatorIndex proto.ColUInt32 + ExchangingMessageFromBlsPubkey proto.ColStr + ExchangingMessageToExecutionAddress route.SafeColFixedStr + ExchangingSignature proto.ColStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlockBlsToExecutionChangeBatch() *canonicalBeaconBlockBlsToExecutionChangeBatch { return &canonicalBeaconBlockBlsToExecutionChangeBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - ExchangingMessageToExecutionAddress: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(42); return c }(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + ExchangingMessageToExecutionAddress: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(42); return c }(), } } @@ -68,58 +39,11 @@ func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) Rows() int { func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) Input() proto.Input { @@ -135,28 +59,7 @@ func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) Input() proto.Input { {Name: "exchanging_message_from_bls_pubkey", Data: &b.ExchangingMessageFromBlsPubkey}, {Name: "exchanging_message_to_execution_address", Data: &b.ExchangingMessageToExecutionAddress}, {Name: "exchanging_signature", Data: &b.ExchangingSignature}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -172,28 +75,7 @@ func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) Reset() { b.ExchangingMessageFromBlsPubkey.Reset() b.ExchangingMessageToExecutionAddress.Reset() b.ExchangingSignature.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -202,7 +84,7 @@ func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) Snapshot() []map[string] out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 33) + row := make(map[string]any, 12) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -214,48 +96,7 @@ func (b *canonicalBeaconBlockBlsToExecutionChangeBatch) Snapshot() []map[string] row["exchanging_message_from_bls_pubkey"] = b.ExchangingMessageFromBlsPubkey.Row(i) row["exchanging_message_to_execution_address"] = string(b.ExchangingMessageToExecutionAddress.Row(i)) row["exchanging_signature"] = b.ExchangingSignature.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change_test.go b/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change_test.go index 53874c06..248dafd5 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_bls_to_execution_change_test.go @@ -28,6 +28,6 @@ func TestSnapshot_canonical_beacon_block_bls_to_execution_change(t *testing.T) { EthV2BeaconBlockBlsToExecutionChange: ðv2.SignedBLSToExecutionChangeV2{}, }, }, 1, map[string]any{ - "meta_client_name": "test-client", + "meta_network_name": "mainnet", }) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go index cffd4cc3..5b644faf 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,54 +11,27 @@ import ( const canonicalBeaconBlockDepositTableName route.TableName = "canonical_beacon_block_deposit" type canonicalBeaconBlockDepositBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockVersion proto.ColStr - DepositProof *proto.ColArr[string] - DepositDataPubkey proto.ColStr - DepositDataWithdrawalCredentials route.SafeColFixedStr - DepositDataAmount proto.ColUInt128 - DepositDataSignature proto.ColStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockVersion proto.ColStr + DepositProof *proto.ColArr[string] + DepositDataPubkey proto.ColStr + DepositDataWithdrawalCredentials route.SafeColFixedStr + DepositDataAmount proto.ColUInt128 + DepositDataSignature proto.ColStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlockDepositBatch() *canonicalBeaconBlockDepositBatch { return &canonicalBeaconBlockDepositBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - DepositProof: proto.NewArray[string](new(proto.ColStr)), - DepositDataWithdrawalCredentials: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + DepositProof: proto.NewArray[string](new(proto.ColStr)), + DepositDataWithdrawalCredentials: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), } } @@ -70,58 +41,11 @@ func (b *canonicalBeaconBlockDepositBatch) Rows() int { func (b *canonicalBeaconBlockDepositBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockDepositBatch) Input() proto.Input { @@ -138,28 +62,7 @@ func (b *canonicalBeaconBlockDepositBatch) Input() proto.Input { {Name: "deposit_data_withdrawal_credentials", Data: &b.DepositDataWithdrawalCredentials}, {Name: "deposit_data_amount", Data: &b.DepositDataAmount}, {Name: "deposit_data_signature", Data: &b.DepositDataSignature}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -176,28 +79,7 @@ func (b *canonicalBeaconBlockDepositBatch) Reset() { b.DepositDataWithdrawalCredentials.Reset() b.DepositDataAmount.Reset() b.DepositDataSignature.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -206,7 +88,7 @@ func (b *canonicalBeaconBlockDepositBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 34) + row := make(map[string]any, 13) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -219,48 +101,7 @@ func (b *canonicalBeaconBlockDepositBatch) Snapshot() []map[string]any { row["deposit_data_withdrawal_credentials"] = string(b.DepositDataWithdrawalCredentials.Row(i)) row["deposit_data_amount"] = route.UInt128ToString(b.DepositDataAmount.Row(i)) row["deposit_data_signature"] = b.DepositDataSignature.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit_test.go b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit_test.go index 31a2b6f6..2a9abaa1 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_deposit_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_deposit_test.go @@ -28,6 +28,6 @@ func TestSnapshot_canonical_beacon_block_deposit(t *testing.T) { EthV2BeaconBlockDeposit: ðv1.DepositV2{}, }, }, 1, map[string]any{ - "meta_client_name": "test-client", + "meta_network_name": "mainnet", }) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go index 2178a773..fff66083 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_execution_transaction.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,75 +11,48 @@ import ( const canonicalBeaconBlockExecutionTransactionTableName route.TableName = "canonical_beacon_block_execution_transaction" type canonicalBeaconBlockExecutionTransactionBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockVersion proto.ColStr - Position proto.ColUInt32 - Hash route.SafeColFixedStr - From route.SafeColFixedStr - To *proto.ColNullable[[]byte] - Nonce proto.ColUInt64 - GasPrice proto.ColUInt128 - Gas proto.ColUInt64 - GasTipCap *proto.ColNullable[proto.UInt128] - GasFeeCap *proto.ColNullable[proto.UInt128] - Value proto.ColUInt128 - Type proto.ColUInt8 - Size proto.ColUInt32 - CallDataSize proto.ColUInt32 - BlobGas *proto.ColNullable[uint64] - BlobGasFeeCap *proto.ColNullable[proto.UInt128] - BlobHashes *proto.ColArr[string] - BlobSidecarsSize *proto.ColNullable[uint32] - BlobSidecarsEmptySize *proto.ColNullable[uint32] - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockVersion proto.ColStr + Position proto.ColUInt32 + Hash route.SafeColFixedStr + From route.SafeColFixedStr + To *proto.ColNullable[[]byte] + Nonce proto.ColUInt64 + GasPrice proto.ColUInt128 + Gas proto.ColUInt64 + GasTipCap *proto.ColNullable[proto.UInt128] + GasFeeCap *proto.ColNullable[proto.UInt128] + Value proto.ColUInt128 + Type proto.ColUInt8 + Size proto.ColUInt32 + CallDataSize proto.ColUInt32 + BlobGas *proto.ColNullable[uint64] + BlobGasFeeCap *proto.ColNullable[proto.UInt128] + BlobHashes *proto.ColArr[string] + BlobSidecarsSize *proto.ColNullable[uint32] + BlobSidecarsEmptySize *proto.ColNullable[uint32] + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlockExecutionTransactionBatch() *canonicalBeaconBlockExecutionTransactionBatch { return &canonicalBeaconBlockExecutionTransactionBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - Hash: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - From: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(42); return c }(), - To: route.NewNullableFixedStr(42), - GasTipCap: new(proto.ColUInt128).Nullable(), - GasFeeCap: new(proto.ColUInt128).Nullable(), - BlobGas: new(proto.ColUInt64).Nullable(), - BlobGasFeeCap: new(proto.ColUInt128).Nullable(), - BlobHashes: proto.NewArray[string](new(proto.ColStr)), - BlobSidecarsSize: new(proto.ColUInt32).Nullable(), - BlobSidecarsEmptySize: new(proto.ColUInt32).Nullable(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + Hash: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + From: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(42); return c }(), + To: route.NewNullableFixedStr(42), + GasTipCap: new(proto.ColUInt128).Nullable(), + GasFeeCap: new(proto.ColUInt128).Nullable(), + BlobGas: new(proto.ColUInt64).Nullable(), + BlobGasFeeCap: new(proto.ColUInt128).Nullable(), + BlobHashes: proto.NewArray[string](new(proto.ColStr)), + BlobSidecarsSize: new(proto.ColUInt32).Nullable(), + BlobSidecarsEmptySize: new(proto.ColUInt32).Nullable(), } } @@ -91,58 +62,11 @@ func (b *canonicalBeaconBlockExecutionTransactionBatch) Rows() int { func (b *canonicalBeaconBlockExecutionTransactionBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockExecutionTransactionBatch) Input() proto.Input { @@ -172,28 +96,7 @@ func (b *canonicalBeaconBlockExecutionTransactionBatch) Input() proto.Input { {Name: "blob_hashes", Data: b.BlobHashes}, {Name: "blob_sidecars_size", Data: b.BlobSidecarsSize}, {Name: "blob_sidecars_empty_size", Data: b.BlobSidecarsEmptySize}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -223,28 +126,7 @@ func (b *canonicalBeaconBlockExecutionTransactionBatch) Reset() { b.BlobHashes.Reset() b.BlobSidecarsSize.Reset() b.BlobSidecarsEmptySize.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -253,7 +135,7 @@ func (b *canonicalBeaconBlockExecutionTransactionBatch) Snapshot() []map[string] out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 47) + row := make(map[string]any, 26) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -307,48 +189,7 @@ func (b *canonicalBeaconBlockExecutionTransactionBatch) Snapshot() []map[string] } else { row["blob_sidecars_empty_size"] = nil } - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go index f3bc9d3e..f23c6f15 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,65 +11,38 @@ import ( const canonicalBeaconBlockProposerSlashingTableName route.TableName = "canonical_beacon_block_proposer_slashing" type canonicalBeaconBlockProposerSlashingBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockVersion proto.ColStr - SignedHeader1MessageSlot proto.ColUInt32 - SignedHeader1MessageProposerIndex proto.ColUInt32 - SignedHeader1MessageBodyRoot route.SafeColFixedStr - SignedHeader1MessageParentRoot route.SafeColFixedStr - SignedHeader1MessageStateRoot route.SafeColFixedStr - SignedHeader1Signature proto.ColStr - SignedHeader2MessageSlot proto.ColUInt32 - SignedHeader2MessageProposerIndex proto.ColUInt32 - SignedHeader2MessageBodyRoot route.SafeColFixedStr - SignedHeader2MessageParentRoot route.SafeColFixedStr - SignedHeader2MessageStateRoot route.SafeColFixedStr - SignedHeader2Signature proto.ColStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockVersion proto.ColStr + SignedHeader1MessageSlot proto.ColUInt32 + SignedHeader1MessageProposerIndex proto.ColUInt32 + SignedHeader1MessageBodyRoot route.SafeColFixedStr + SignedHeader1MessageParentRoot route.SafeColFixedStr + SignedHeader1MessageStateRoot route.SafeColFixedStr + SignedHeader1Signature proto.ColStr + SignedHeader2MessageSlot proto.ColUInt32 + SignedHeader2MessageProposerIndex proto.ColUInt32 + SignedHeader2MessageBodyRoot route.SafeColFixedStr + SignedHeader2MessageParentRoot route.SafeColFixedStr + SignedHeader2MessageStateRoot route.SafeColFixedStr + SignedHeader2Signature proto.ColStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlockProposerSlashingBatch() *canonicalBeaconBlockProposerSlashingBatch { return &canonicalBeaconBlockProposerSlashingBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - SignedHeader1MessageBodyRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - SignedHeader1MessageParentRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - SignedHeader1MessageStateRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - SignedHeader2MessageBodyRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - SignedHeader2MessageParentRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - SignedHeader2MessageStateRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + SignedHeader1MessageBodyRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + SignedHeader1MessageParentRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + SignedHeader1MessageStateRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + SignedHeader2MessageBodyRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + SignedHeader2MessageParentRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + SignedHeader2MessageStateRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), } } @@ -81,58 +52,11 @@ func (b *canonicalBeaconBlockProposerSlashingBatch) Rows() int { func (b *canonicalBeaconBlockProposerSlashingBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockProposerSlashingBatch) Input() proto.Input { @@ -156,28 +80,7 @@ func (b *canonicalBeaconBlockProposerSlashingBatch) Input() proto.Input { {Name: "signed_header_2_message_parent_root", Data: &b.SignedHeader2MessageParentRoot}, {Name: "signed_header_2_message_state_root", Data: &b.SignedHeader2MessageStateRoot}, {Name: "signed_header_2_signature", Data: &b.SignedHeader2Signature}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -201,28 +104,7 @@ func (b *canonicalBeaconBlockProposerSlashingBatch) Reset() { b.SignedHeader2MessageParentRoot.Reset() b.SignedHeader2MessageStateRoot.Reset() b.SignedHeader2Signature.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -231,7 +113,7 @@ func (b *canonicalBeaconBlockProposerSlashingBatch) Snapshot() []map[string]any out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 41) + row := make(map[string]any, 20) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -251,48 +133,7 @@ func (b *canonicalBeaconBlockProposerSlashingBatch) Snapshot() []map[string]any row["signed_header_2_message_parent_root"] = string(b.SignedHeader2MessageParentRoot.Row(i)) row["signed_header_2_message_state_root"] = string(b.SignedHeader2MessageStateRoot.Row(i)) row["signed_header_2_signature"] = b.SignedHeader2Signature.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing_test.go b/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing_test.go index 3217757e..dd8f84f9 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_proposer_slashing_test.go @@ -28,6 +28,6 @@ func TestSnapshot_canonical_beacon_block_proposer_slashing(t *testing.T) { EthV2BeaconBlockProposerSlashing: ðv1.ProposerSlashingV2{}, }, }, 1, map[string]any{ - "meta_client_name": "test-client", + "meta_network_name": "mainnet", }) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go index 35c1436a..eaeb0e9c 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,57 +11,30 @@ import ( const canonicalBeaconBlockSyncAggregateTableName route.TableName = "canonical_beacon_block_sync_aggregate" type canonicalBeaconBlockSyncAggregateBatch struct { - UpdatedDateTime proto.ColDateTime - EventDateTime proto.ColDateTime64 - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockVersion proto.ColStr - SyncCommitteePeriod proto.ColUInt64 - SyncCommitteeBits proto.ColStr - SyncCommitteeSignature proto.ColStr - ValidatorsParticipated *proto.ColArr[uint32] - ValidatorsMissed *proto.ColArr[uint32] - ParticipationCount proto.ColUInt16 - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + EventDateTime proto.ColDateTime64 + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockVersion proto.ColStr + SyncCommitteePeriod proto.ColUInt64 + SyncCommitteeBits proto.ColStr + SyncCommitteeSignature proto.ColStr + ValidatorsParticipated *proto.ColArr[uint32] + ValidatorsMissed *proto.ColArr[uint32] + ParticipationCount proto.ColUInt16 + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlockSyncAggregateBatch() *canonicalBeaconBlockSyncAggregateBatch { return &canonicalBeaconBlockSyncAggregateBatch{ - EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - ValidatorsParticipated: proto.NewArray[uint32](new(proto.ColUInt32)), - ValidatorsMissed: proto.NewArray[uint32](new(proto.ColUInt32)), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + ValidatorsParticipated: proto.NewArray[uint32](new(proto.ColUInt32)), + ValidatorsMissed: proto.NewArray[uint32](new(proto.ColUInt32)), } } @@ -73,58 +44,11 @@ func (b *canonicalBeaconBlockSyncAggregateBatch) Rows() int { func (b *canonicalBeaconBlockSyncAggregateBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockSyncAggregateBatch) Input() proto.Input { @@ -143,28 +67,7 @@ func (b *canonicalBeaconBlockSyncAggregateBatch) Input() proto.Input { {Name: "validators_participated", Data: b.ValidatorsParticipated}, {Name: "validators_missed", Data: b.ValidatorsMissed}, {Name: "participation_count", Data: &b.ParticipationCount}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -183,28 +86,7 @@ func (b *canonicalBeaconBlockSyncAggregateBatch) Reset() { b.ValidatorsParticipated.Reset() b.ValidatorsMissed.Reset() b.ParticipationCount.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -213,7 +95,7 @@ func (b *canonicalBeaconBlockSyncAggregateBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 36) + row := make(map[string]any, 15) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -228,48 +110,7 @@ func (b *canonicalBeaconBlockSyncAggregateBatch) Snapshot() []map[string]any { row["validators_participated"] = b.ValidatorsParticipated.Row(i) row["validators_missed"] = b.ValidatorsMissed.Row(i) row["participation_count"] = b.ParticipationCount.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate_test.go b/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate_test.go index 33f77736..bd4e6ab1 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_sync_aggregate_test.go @@ -22,6 +22,6 @@ func TestSnapshot_canonical_beacon_block_sync_aggregate(t *testing.T) { }, }, }, 1, map[string]any{ - "meta_client_name": "test-client", + "meta_network_name": "mainnet", }) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go index 67b6e6f5..56c22003 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,50 +11,23 @@ import ( const canonicalBeaconBlockVoluntaryExitTableName route.TableName = "canonical_beacon_block_voluntary_exit" type canonicalBeaconBlockVoluntaryExitBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockVersion proto.ColStr - VoluntaryExitMessageEpoch proto.ColUInt32 - VoluntaryExitMessageValidatorIndex proto.ColUInt32 - VoluntaryExitSignature proto.ColStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockVersion proto.ColStr + VoluntaryExitMessageEpoch proto.ColUInt32 + VoluntaryExitMessageValidatorIndex proto.ColUInt32 + VoluntaryExitSignature proto.ColStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlockVoluntaryExitBatch() *canonicalBeaconBlockVoluntaryExitBatch { return &canonicalBeaconBlockVoluntaryExitBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), } } @@ -66,58 +37,11 @@ func (b *canonicalBeaconBlockVoluntaryExitBatch) Rows() int { func (b *canonicalBeaconBlockVoluntaryExitBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockVoluntaryExitBatch) Input() proto.Input { @@ -132,28 +56,7 @@ func (b *canonicalBeaconBlockVoluntaryExitBatch) Input() proto.Input { {Name: "voluntary_exit_message_epoch", Data: &b.VoluntaryExitMessageEpoch}, {Name: "voluntary_exit_message_validator_index", Data: &b.VoluntaryExitMessageValidatorIndex}, {Name: "voluntary_exit_signature", Data: &b.VoluntaryExitSignature}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -168,28 +71,7 @@ func (b *canonicalBeaconBlockVoluntaryExitBatch) Reset() { b.VoluntaryExitMessageEpoch.Reset() b.VoluntaryExitMessageValidatorIndex.Reset() b.VoluntaryExitSignature.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -198,7 +80,7 @@ func (b *canonicalBeaconBlockVoluntaryExitBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 32) + row := make(map[string]any, 11) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -209,48 +91,7 @@ func (b *canonicalBeaconBlockVoluntaryExitBatch) Snapshot() []map[string]any { row["voluntary_exit_message_epoch"] = b.VoluntaryExitMessageEpoch.Row(i) row["voluntary_exit_message_validator_index"] = b.VoluntaryExitMessageValidatorIndex.Row(i) row["voluntary_exit_signature"] = b.VoluntaryExitSignature.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit_test.go b/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit_test.go index dd1f188c..4ef814f4 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_voluntary_exit_test.go @@ -28,6 +28,6 @@ func TestSnapshot_canonical_beacon_block_voluntary_exit(t *testing.T) { EthV2BeaconBlockVoluntaryExit: ðv1.SignedVoluntaryExitV2{}, }, }, 1, map[string]any{ - "meta_client_name": "test-client", + "meta_network_name": "mainnet", }) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go index c4a01ff6..0c97debc 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_block_withdrawal.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,52 +11,25 @@ import ( const canonicalBeaconBlockWithdrawalTableName route.TableName = "canonical_beacon_block_withdrawal" type canonicalBeaconBlockWithdrawalBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - BlockRoot route.SafeColFixedStr - BlockVersion proto.ColStr - WithdrawalIndex proto.ColUInt32 - WithdrawalValidatorIndex proto.ColUInt32 - WithdrawalAddress route.SafeColFixedStr - WithdrawalAmount proto.ColUInt128 - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + BlockRoot route.SafeColFixedStr + BlockVersion proto.ColStr + WithdrawalIndex proto.ColUInt32 + WithdrawalValidatorIndex proto.ColUInt32 + WithdrawalAddress route.SafeColFixedStr + WithdrawalAmount proto.ColUInt128 + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconBlockWithdrawalBatch() *canonicalBeaconBlockWithdrawalBatch { return &canonicalBeaconBlockWithdrawalBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - WithdrawalAddress: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(42); return c }(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + WithdrawalAddress: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(42); return c }(), } } @@ -68,58 +39,11 @@ func (b *canonicalBeaconBlockWithdrawalBatch) Rows() int { func (b *canonicalBeaconBlockWithdrawalBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconBlockWithdrawalBatch) Input() proto.Input { @@ -135,28 +59,7 @@ func (b *canonicalBeaconBlockWithdrawalBatch) Input() proto.Input { {Name: "withdrawal_validator_index", Data: &b.WithdrawalValidatorIndex}, {Name: "withdrawal_address", Data: &b.WithdrawalAddress}, {Name: "withdrawal_amount", Data: &b.WithdrawalAmount}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -172,28 +75,7 @@ func (b *canonicalBeaconBlockWithdrawalBatch) Reset() { b.WithdrawalValidatorIndex.Reset() b.WithdrawalAddress.Reset() b.WithdrawalAmount.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -202,7 +84,7 @@ func (b *canonicalBeaconBlockWithdrawalBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 33) + row := make(map[string]any, 12) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -214,48 +96,7 @@ func (b *canonicalBeaconBlockWithdrawalBatch) Snapshot() []map[string]any { row["withdrawal_validator_index"] = b.WithdrawalValidatorIndex.Row(i) row["withdrawal_address"] = string(b.WithdrawalAddress.Row(i)) row["withdrawal_amount"] = route.UInt128ToString(b.WithdrawalAmount.Row(i)) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go index d5e7fd46..89ee8c8c 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_committee.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,47 +11,20 @@ import ( const canonicalBeaconCommitteeTableName route.TableName = "canonical_beacon_committee" type canonicalBeaconCommitteeBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - CommitteeIndex proto.ColStr - Validators *proto.ColArr[uint32] - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + CommitteeIndex proto.ColStr + Validators *proto.ColArr[uint32] + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconCommitteeBatch() *canonicalBeaconCommitteeBatch { return &canonicalBeaconCommitteeBatch{ - Validators: proto.NewArray[uint32](new(proto.ColUInt32)), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + Validators: proto.NewArray[uint32](new(proto.ColUInt32)), } } @@ -63,58 +34,11 @@ func (b *canonicalBeaconCommitteeBatch) Rows() int { func (b *canonicalBeaconCommitteeBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconCommitteeBatch) Input() proto.Input { @@ -126,28 +50,7 @@ func (b *canonicalBeaconCommitteeBatch) Input() proto.Input { {Name: "validators", Data: b.Validators}, {Name: "epoch", Data: &b.Epoch}, {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -159,28 +62,7 @@ func (b *canonicalBeaconCommitteeBatch) Reset() { b.Validators.Reset() b.Epoch.Reset() b.EpochStartDateTime.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -189,7 +71,7 @@ func (b *canonicalBeaconCommitteeBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 29) + row := make(map[string]any, 8) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -197,48 +79,7 @@ func (b *canonicalBeaconCommitteeBatch) Snapshot() []map[string]any { row["validators"] = b.Validators.Row(i) row["epoch"] = b.Epoch.Row(i) row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go index 2abaca7a..6179d9c9 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,64 +11,37 @@ import ( const canonicalBeaconElaboratedAttestationTableName route.TableName = "canonical_beacon_elaborated_attestation" type canonicalBeaconElaboratedAttestationBatch struct { - UpdatedDateTime proto.ColDateTime - BlockSlot proto.ColUInt32 - BlockSlotStartDateTime proto.ColDateTime - BlockEpoch proto.ColUInt32 - BlockEpochStartDateTime proto.ColDateTime - PositionInBlock proto.ColUInt32 - BlockRoot route.SafeColFixedStr - Validators *proto.ColArr[uint32] - CommitteeIndex proto.ColStr - BeaconBlockRoot route.SafeColFixedStr - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - SourceEpoch proto.ColUInt32 - SourceEpochStartDateTime proto.ColDateTime - SourceRoot route.SafeColFixedStr - TargetEpoch proto.ColUInt32 - TargetEpochStartDateTime proto.ColDateTime - TargetRoot route.SafeColFixedStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + BlockSlot proto.ColUInt32 + BlockSlotStartDateTime proto.ColDateTime + BlockEpoch proto.ColUInt32 + BlockEpochStartDateTime proto.ColDateTime + PositionInBlock proto.ColUInt32 + BlockRoot route.SafeColFixedStr + Validators *proto.ColArr[uint32] + CommitteeIndex proto.ColStr + BeaconBlockRoot route.SafeColFixedStr + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + SourceEpoch proto.ColUInt32 + SourceEpochStartDateTime proto.ColDateTime + SourceRoot route.SafeColFixedStr + TargetEpoch proto.ColUInt32 + TargetEpochStartDateTime proto.ColDateTime + TargetRoot route.SafeColFixedStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconElaboratedAttestationBatch() *canonicalBeaconElaboratedAttestationBatch { return &canonicalBeaconElaboratedAttestationBatch{ - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - Validators: proto.NewArray[uint32](new(proto.ColUInt32)), - BeaconBlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - SourceRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - TargetRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + Validators: proto.NewArray[uint32](new(proto.ColUInt32)), + BeaconBlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + SourceRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), + TargetRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), } } @@ -80,58 +51,11 @@ func (b *canonicalBeaconElaboratedAttestationBatch) Rows() int { func (b *canonicalBeaconElaboratedAttestationBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconElaboratedAttestationBatch) Input() proto.Input { @@ -156,28 +80,7 @@ func (b *canonicalBeaconElaboratedAttestationBatch) Input() proto.Input { {Name: "target_epoch", Data: &b.TargetEpoch}, {Name: "target_epoch_start_date_time", Data: &b.TargetEpochStartDateTime}, {Name: "target_root", Data: &b.TargetRoot}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -202,28 +105,7 @@ func (b *canonicalBeaconElaboratedAttestationBatch) Reset() { b.TargetEpoch.Reset() b.TargetEpochStartDateTime.Reset() b.TargetRoot.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -232,7 +114,7 @@ func (b *canonicalBeaconElaboratedAttestationBatch) Snapshot() []map[string]any out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 42) + row := make(map[string]any, 21) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["block_slot"] = b.BlockSlot.Row(i) row["block_slot_start_date_time"] = b.BlockSlotStartDateTime.Row(i).Unix() @@ -253,48 +135,7 @@ func (b *canonicalBeaconElaboratedAttestationBatch) Snapshot() []map[string]any row["target_epoch"] = b.TargetEpoch.Row(i) row["target_epoch_start_date_time"] = b.TargetEpochStartDateTime.Row(i).Unix() row["target_root"] = string(b.TargetRoot.Row(i)) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation_test.go b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation_test.go index 8ed75518..f8e90a75 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_elaborated_attestation_test.go @@ -32,6 +32,6 @@ func TestSnapshot_canonical_beacon_elaborated_attestation(t *testing.T) { }, }, }, 1, map[string]any{ - "meta_client_name": "test-client", + "meta_network_name": "mainnet", }) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go index 7417cbb0..49410d7a 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_proposer_duty.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,47 +11,19 @@ import ( const canonicalBeaconProposerDutyTableName route.TableName = "canonical_beacon_proposer_duty" type canonicalBeaconProposerDutyBatch struct { - UpdatedDateTime proto.ColDateTime - Slot proto.ColUInt32 - SlotStartDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - ProposerValidatorIndex proto.ColUInt32 - ProposerPubkey proto.ColStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Slot proto.ColUInt32 + SlotStartDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + ProposerValidatorIndex proto.ColUInt32 + ProposerPubkey proto.ColStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconProposerDutyBatch() *canonicalBeaconProposerDutyBatch { - return &canonicalBeaconProposerDutyBatch{ - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), - } + return &canonicalBeaconProposerDutyBatch{} } func (b *canonicalBeaconProposerDutyBatch) Rows() int { @@ -62,58 +32,11 @@ func (b *canonicalBeaconProposerDutyBatch) Rows() int { func (b *canonicalBeaconProposerDutyBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconProposerDutyBatch) Input() proto.Input { @@ -125,28 +48,7 @@ func (b *canonicalBeaconProposerDutyBatch) Input() proto.Input { {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "proposer_validator_index", Data: &b.ProposerValidatorIndex}, {Name: "proposer_pubkey", Data: &b.ProposerPubkey}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -158,28 +60,7 @@ func (b *canonicalBeaconProposerDutyBatch) Reset() { b.EpochStartDateTime.Reset() b.ProposerValidatorIndex.Reset() b.ProposerPubkey.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -188,7 +69,7 @@ func (b *canonicalBeaconProposerDutyBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 29) + row := make(map[string]any, 8) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -196,48 +77,7 @@ func (b *canonicalBeaconProposerDutyBatch) Snapshot() []map[string]any { row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["proposer_validator_index"] = b.ProposerValidatorIndex.Row(i) row["proposer_pubkey"] = b.ProposerPubkey.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go index 8133e591..4f532410 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_sync_committee.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,47 +11,20 @@ import ( const canonicalBeaconSyncCommitteeTableName route.TableName = "canonical_beacon_sync_committee" type canonicalBeaconSyncCommitteeBatch struct { - UpdatedDateTime proto.ColDateTime - EventDateTime proto.ColDateTime64 - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - SyncCommitteePeriod proto.ColUInt64 - ValidatorAggregates *proto.ColArr[[]uint32] - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + EventDateTime proto.ColDateTime64 + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + SyncCommitteePeriod proto.ColUInt64 + ValidatorAggregates *proto.ColArr[[]uint32] + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconSyncCommitteeBatch() *canonicalBeaconSyncCommitteeBatch { return &canonicalBeaconSyncCommitteeBatch{ - EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), - ValidatorAggregates: proto.NewArray[[]uint32](&proto.ColArr[uint32]{Data: new(proto.ColUInt32)}), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), + ValidatorAggregates: proto.NewArray[[]uint32](&proto.ColArr[uint32]{Data: new(proto.ColUInt32)}), } } @@ -63,58 +34,11 @@ func (b *canonicalBeaconSyncCommitteeBatch) Rows() int { func (b *canonicalBeaconSyncCommitteeBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconSyncCommitteeBatch) Input() proto.Input { @@ -125,28 +49,7 @@ func (b *canonicalBeaconSyncCommitteeBatch) Input() proto.Input { {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "sync_committee_period", Data: &b.SyncCommitteePeriod}, {Name: "validator_aggregates", Data: b.ValidatorAggregates}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -157,28 +60,7 @@ func (b *canonicalBeaconSyncCommitteeBatch) Reset() { b.EpochStartDateTime.Reset() b.SyncCommitteePeriod.Reset() b.ValidatorAggregates.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -187,55 +69,14 @@ func (b *canonicalBeaconSyncCommitteeBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 28) + row := make(map[string]any, 7) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["epoch"] = b.Epoch.Row(i) row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["sync_committee_period"] = b.SyncCommitteePeriod.Row(i) row["validator_aggregates"] = b.ValidatorAggregates.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_sync_committee_test.go b/pkg/consumoor/route/canonical/canonical_beacon_sync_committee_test.go index 49475335..5f440202 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_sync_committee_test.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_sync_committee_test.go @@ -37,6 +37,6 @@ func TestSnapshot_canonical_beacon_sync_committee(t *testing.T) { }, }, }, 1, map[string]any{ - "meta_client_name": "test-client", + "meta_network_name": "mainnet", }) } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go index f271bcdb..f7866f56 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,57 +11,30 @@ import ( const canonicalBeaconValidatorsTableName route.TableName = "canonical_beacon_validators" type canonicalBeaconValidatorsBatch struct { - UpdatedDateTime proto.ColDateTime - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - Index proto.ColUInt32 - Balance *proto.ColNullable[uint64] - Status proto.ColStr - EffectiveBalance *proto.ColNullable[uint64] - Slashed proto.ColBool - ActivationEpoch *proto.ColNullable[uint64] - ActivationEligibilityEpoch *proto.ColNullable[uint64] - ExitEpoch *proto.ColNullable[uint64] - WithdrawableEpoch *proto.ColNullable[uint64] - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + Index proto.ColUInt32 + Balance *proto.ColNullable[uint64] + Status proto.ColStr + EffectiveBalance *proto.ColNullable[uint64] + Slashed proto.ColBool + ActivationEpoch *proto.ColNullable[uint64] + ActivationEligibilityEpoch *proto.ColNullable[uint64] + ExitEpoch *proto.ColNullable[uint64] + WithdrawableEpoch *proto.ColNullable[uint64] + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconValidatorsBatch() *canonicalBeaconValidatorsBatch { return &canonicalBeaconValidatorsBatch{ - Balance: new(proto.ColUInt64).Nullable(), - EffectiveBalance: new(proto.ColUInt64).Nullable(), - ActivationEpoch: new(proto.ColUInt64).Nullable(), - ActivationEligibilityEpoch: new(proto.ColUInt64).Nullable(), - ExitEpoch: new(proto.ColUInt64).Nullable(), - WithdrawableEpoch: new(proto.ColUInt64).Nullable(), - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), + Balance: new(proto.ColUInt64).Nullable(), + EffectiveBalance: new(proto.ColUInt64).Nullable(), + ActivationEpoch: new(proto.ColUInt64).Nullable(), + ActivationEligibilityEpoch: new(proto.ColUInt64).Nullable(), + ExitEpoch: new(proto.ColUInt64).Nullable(), + WithdrawableEpoch: new(proto.ColUInt64).Nullable(), } } @@ -73,58 +44,11 @@ func (b *canonicalBeaconValidatorsBatch) Rows() int { func (b *canonicalBeaconValidatorsBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconValidatorsBatch) Input() proto.Input { @@ -141,28 +65,7 @@ func (b *canonicalBeaconValidatorsBatch) Input() proto.Input { {Name: "activation_eligibility_epoch", Data: b.ActivationEligibilityEpoch}, {Name: "exit_epoch", Data: b.ExitEpoch}, {Name: "withdrawable_epoch", Data: b.WithdrawableEpoch}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -179,28 +82,7 @@ func (b *canonicalBeaconValidatorsBatch) Reset() { b.ActivationEligibilityEpoch.Reset() b.ExitEpoch.Reset() b.WithdrawableEpoch.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -209,7 +91,7 @@ func (b *canonicalBeaconValidatorsBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 34) + row := make(map[string]any, 13) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["epoch"] = b.Epoch.Row(i) row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() @@ -246,48 +128,7 @@ func (b *canonicalBeaconValidatorsBatch) Snapshot() []map[string]any { } else { row["withdrawable_epoch"] = nil } - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go index ecaf43c8..11ddeb7d 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,46 +11,17 @@ import ( const canonicalBeaconValidatorsPubkeysTableName route.TableName = "canonical_beacon_validators_pubkeys" type canonicalBeaconValidatorsPubkeysBatch struct { - UpdatedDateTime proto.ColDateTime - Version proto.ColUInt32 - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - Index proto.ColUInt32 - Pubkey proto.ColStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + Index proto.ColUInt32 + Pubkey proto.ColStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconValidatorsPubkeysBatch() *canonicalBeaconValidatorsPubkeysBatch { - return &canonicalBeaconValidatorsPubkeysBatch{ - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), - } + return &canonicalBeaconValidatorsPubkeysBatch{} } func (b *canonicalBeaconValidatorsPubkeysBatch) Rows() int { @@ -61,122 +30,31 @@ func (b *canonicalBeaconValidatorsPubkeysBatch) Rows() int { func (b *canonicalBeaconValidatorsPubkeysBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconValidatorsPubkeysBatch) Input() proto.Input { return proto.Input{ {Name: "updated_date_time", Data: &b.UpdatedDateTime}, - {Name: "version", Data: &b.Version}, {Name: "epoch", Data: &b.Epoch}, {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "index", Data: &b.Index}, {Name: "pubkey", Data: &b.Pubkey}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } func (b *canonicalBeaconValidatorsPubkeysBatch) Reset() { b.UpdatedDateTime.Reset() - b.Version.Reset() b.Epoch.Reset() b.EpochStartDateTime.Reset() b.Index.Reset() b.Pubkey.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -185,55 +63,13 @@ func (b *canonicalBeaconValidatorsPubkeysBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 28) + row := make(map[string]any, 6) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() - row["version"] = b.Version.Row(i) row["epoch"] = b.Epoch.Row(i) row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["index"] = b.Index.Row(i) row["pubkey"] = b.Pubkey.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.go b/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.go index a661e345..185407d9 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators_pubkeys.go @@ -74,7 +74,6 @@ func (b *canonicalBeaconValidatorsPubkeysBatch) FlattenTo(event *xatu.DecoratedE } b.UpdatedDateTime.Append(now) - b.Version.Append(uint32(4294967295 - epochStartTime.Unix())) //nolint:gosec // inverse timestamp for ReplacingMergeTree dedup b.Epoch.Append(epoch) b.EpochStartDateTime.Append(epochStartTime) diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go b/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go index aed0b34d..6c424ac5 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.gen.go @@ -3,8 +3,6 @@ package canonical import ( - "net" - "github.com/ClickHouse/ch-go/proto" "github.com/ethpandaops/xatu/pkg/consumoor/route" "github.com/ethpandaops/xatu/pkg/proto/xatu" @@ -13,46 +11,17 @@ import ( const canonicalBeaconValidatorsWithdrawalCredentialsTableName route.TableName = "canonical_beacon_validators_withdrawal_credentials" type canonicalBeaconValidatorsWithdrawalCredentialsBatch struct { - UpdatedDateTime proto.ColDateTime - Version proto.ColUInt32 - Epoch proto.ColUInt32 - EpochStartDateTime proto.ColDateTime - Index proto.ColUInt32 - WithdrawalCredentials proto.ColStr - MetaClientName proto.ColStr - MetaClientID proto.ColStr - MetaClientVersion proto.ColStr - MetaClientImplementation proto.ColStr - MetaClientOS proto.ColStr - MetaClientIP *proto.ColNullable[proto.IPv6] - MetaClientGeoCity proto.ColStr - MetaClientGeoCountry proto.ColStr - MetaClientGeoCountryCode proto.ColStr - MetaClientGeoContinentCode proto.ColStr - MetaClientGeoLongitude *proto.ColNullable[float64] - MetaClientGeoLatitude *proto.ColNullable[float64] - MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] - MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 - MetaNetworkName proto.ColStr - MetaConsensusVersion proto.ColStr - MetaConsensusVersionMajor proto.ColStr - MetaConsensusVersionMinor proto.ColStr - MetaConsensusVersionPatch proto.ColStr - MetaConsensusImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] - rows int + UpdatedDateTime proto.ColDateTime + Epoch proto.ColUInt32 + EpochStartDateTime proto.ColDateTime + Index proto.ColUInt32 + WithdrawalCredentials proto.ColStr + MetaNetworkName proto.ColStr + rows int } func newcanonicalBeaconValidatorsWithdrawalCredentialsBatch() *canonicalBeaconValidatorsWithdrawalCredentialsBatch { - return &canonicalBeaconValidatorsWithdrawalCredentialsBatch{ - MetaClientIP: new(proto.ColIPv6).Nullable(), - MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), - MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), - MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), - } + return &canonicalBeaconValidatorsWithdrawalCredentialsBatch{} } func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) Rows() int { @@ -61,122 +30,31 @@ func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) Rows() int { func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaClientName.Append("") - b.MetaClientID.Append("") - b.MetaClientVersion.Append("") - b.MetaClientImplementation.Append("") - b.MetaClientOS.Append("") - b.MetaClientIP.Append(proto.Nullable[proto.IPv6]{}) - b.MetaClientGeoCity.Append("") - b.MetaClientGeoCountry.Append("") - b.MetaClientGeoCountryCode.Append("") - b.MetaClientGeoContinentCode.Append("") - b.MetaClientGeoLongitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaConsensusVersion.Append("") - b.MetaConsensusVersionMajor.Append("") - b.MetaConsensusVersionMinor.Append("") - b.MetaConsensusVersionPatch.Append("") - b.MetaConsensusImplementation.Append("") - b.MetaLabels.Append(nil) return } - b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) - b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) - b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) - b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) - b.MetaClientIP.Append(route.NormalizeIPToIPv6Nullable(event.GetMeta().GetServer().GetClient().GetIP())) - b.MetaClientGeoCity.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCity()) - b.MetaClientGeoCountry.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountry()) - b.MetaClientGeoCountryCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetCountryCode()) - b.MetaClientGeoContinentCode.Append(event.GetMeta().GetServer().GetClient().GetGeo().GetContinentCode()) - b.MetaClientGeoLongitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLongitude())) - b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) - b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) - b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - cvNorm, cvMajor, cvMinor, cvPatch := route.ParseConsensusVersion(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetVersion()) - b.MetaConsensusVersion.Append(cvNorm) - b.MetaConsensusVersionMajor.Append(cvMajor) - b.MetaConsensusVersionMinor.Append(cvMinor) - b.MetaConsensusVersionPatch.Append(cvPatch) - b.MetaConsensusImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetConsensus().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) Input() proto.Input { return proto.Input{ {Name: "updated_date_time", Data: &b.UpdatedDateTime}, - {Name: "version", Data: &b.Version}, {Name: "epoch", Data: &b.Epoch}, {Name: "epoch_start_date_time", Data: &b.EpochStartDateTime}, {Name: "index", Data: &b.Index}, {Name: "withdrawal_credentials", Data: &b.WithdrawalCredentials}, - {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, - {Name: "meta_client_version", Data: &b.MetaClientVersion}, - {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, - {Name: "meta_client_os", Data: &b.MetaClientOS}, - {Name: "meta_client_ip", Data: b.MetaClientIP}, - {Name: "meta_client_geo_city", Data: &b.MetaClientGeoCity}, - {Name: "meta_client_geo_country", Data: &b.MetaClientGeoCountry}, - {Name: "meta_client_geo_country_code", Data: &b.MetaClientGeoCountryCode}, - {Name: "meta_client_geo_continent_code", Data: &b.MetaClientGeoContinentCode}, - {Name: "meta_client_geo_longitude", Data: b.MetaClientGeoLongitude}, - {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, - {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, - {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_consensus_version", Data: &b.MetaConsensusVersion}, - {Name: "meta_consensus_version_major", Data: &b.MetaConsensusVersionMajor}, - {Name: "meta_consensus_version_minor", Data: &b.MetaConsensusVersionMinor}, - {Name: "meta_consensus_version_patch", Data: &b.MetaConsensusVersionPatch}, - {Name: "meta_consensus_implementation", Data: &b.MetaConsensusImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) Reset() { b.UpdatedDateTime.Reset() - b.Version.Reset() b.Epoch.Reset() b.EpochStartDateTime.Reset() b.Index.Reset() b.WithdrawalCredentials.Reset() - b.MetaClientName.Reset() - b.MetaClientID.Reset() - b.MetaClientVersion.Reset() - b.MetaClientImplementation.Reset() - b.MetaClientOS.Reset() - b.MetaClientIP.Reset() - b.MetaClientGeoCity.Reset() - b.MetaClientGeoCountry.Reset() - b.MetaClientGeoCountryCode.Reset() - b.MetaClientGeoContinentCode.Reset() - b.MetaClientGeoLongitude.Reset() - b.MetaClientGeoLatitude.Reset() - b.MetaClientGeoAutonomousSystemNumber.Reset() - b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaConsensusVersion.Reset() - b.MetaConsensusVersionMajor.Reset() - b.MetaConsensusVersionMinor.Reset() - b.MetaConsensusVersionPatch.Reset() - b.MetaConsensusImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -185,55 +63,13 @@ func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) Snapshot() []map[s out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 28) + row := make(map[string]any, 6) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() - row["version"] = b.Version.Row(i) row["epoch"] = b.Epoch.Row(i) row["epoch_start_date_time"] = b.EpochStartDateTime.Row(i).Unix() row["index"] = b.Index.Row(i) row["withdrawal_credentials"] = b.WithdrawalCredentials.Row(i) - row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) - row["meta_client_version"] = b.MetaClientVersion.Row(i) - row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) - row["meta_client_os"] = b.MetaClientOS.Row(i) - if v := b.MetaClientIP.Row(i); v.Set { - row["meta_client_ip"] = net.IP(v.Value[:]).String() - } else { - row["meta_client_ip"] = nil - } - row["meta_client_geo_city"] = b.MetaClientGeoCity.Row(i) - row["meta_client_geo_country"] = b.MetaClientGeoCountry.Row(i) - row["meta_client_geo_country_code"] = b.MetaClientGeoCountryCode.Row(i) - row["meta_client_geo_continent_code"] = b.MetaClientGeoContinentCode.Row(i) - if v := b.MetaClientGeoLongitude.Row(i); v.Set { - row["meta_client_geo_longitude"] = v.Value - } else { - row["meta_client_geo_longitude"] = nil - } - if v := b.MetaClientGeoLatitude.Row(i); v.Set { - row["meta_client_geo_latitude"] = v.Value - } else { - row["meta_client_geo_latitude"] = nil - } - if v := b.MetaClientGeoAutonomousSystemNumber.Row(i); v.Set { - row["meta_client_geo_autonomous_system_number"] = v.Value - } else { - row["meta_client_geo_autonomous_system_number"] = nil - } - if v := b.MetaClientGeoAutonomousSystemOrganization.Row(i); v.Set { - row["meta_client_geo_autonomous_system_organization"] = v.Value - } else { - row["meta_client_geo_autonomous_system_organization"] = nil - } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_consensus_version"] = b.MetaConsensusVersion.Row(i) - row["meta_consensus_version_major"] = b.MetaConsensusVersionMajor.Row(i) - row["meta_consensus_version_minor"] = b.MetaConsensusVersionMinor.Row(i) - row["meta_consensus_version_patch"] = b.MetaConsensusVersionPatch.Row(i) - row["meta_consensus_implementation"] = b.MetaConsensusImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.go b/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.go index 08a5ec23..7472fce5 100644 --- a/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.go +++ b/pkg/consumoor/route/canonical/canonical_beacon_validators_withdrawal_credentials.go @@ -74,7 +74,6 @@ func (b *canonicalBeaconValidatorsWithdrawalCredentialsBatch) FlattenTo(event *x } b.UpdatedDateTime.Append(now) - b.Version.Append(uint32(4294967295 - epochStartTime.Unix())) //nolint:gosec // inverse timestamp for ReplacingMergeTree dedup b.Epoch.Append(epoch) b.EpochStartDateTime.Append(epochStartTime) diff --git a/pkg/consumoor/route/execution/consensus_engine_api_get_blobs.gen.go b/pkg/consumoor/route/execution/consensus_engine_api_get_blobs.gen.go index 7ca229f8..39b3d7e8 100644 --- a/pkg/consumoor/route/execution/consensus_engine_api_get_blobs.gen.go +++ b/pkg/consumoor/route/execution/consensus_engine_api_get_blobs.gen.go @@ -35,7 +35,6 @@ type consensusEngineApiGetBlobsBatch struct { MetaExecutionVersionMinor proto.ColStr MetaExecutionVersionPatch proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -48,7 +47,6 @@ type consensusEngineApiGetBlobsBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -85,7 +83,6 @@ func (b *consensusEngineApiGetBlobsBatch) appendMetadata(event *xatu.DecoratedEv b.MetaExecutionVersionMinor.Append("") b.MetaExecutionVersionPatch.Append("") b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -98,7 +95,6 @@ func (b *consensusEngineApiGetBlobsBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } @@ -109,7 +105,6 @@ func (b *consensusEngineApiGetBlobsBatch) appendMetadata(event *xatu.DecoratedEv b.MetaExecutionVersionMinor.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionMinor()) b.MetaExecutionVersionPatch.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionPatch()) b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -122,7 +117,6 @@ func (b *consensusEngineApiGetBlobsBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -150,7 +144,6 @@ func (b *consensusEngineApiGetBlobsBatch) Input() proto.Input { {Name: "meta_execution_version_minor", Data: &b.MetaExecutionVersionMinor}, {Name: "meta_execution_version_patch", Data: &b.MetaExecutionVersionPatch}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -163,7 +156,6 @@ func (b *consensusEngineApiGetBlobsBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -191,7 +183,6 @@ func (b *consensusEngineApiGetBlobsBatch) Reset() { b.MetaExecutionVersionMinor.Reset() b.MetaExecutionVersionPatch.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -204,7 +195,6 @@ func (b *consensusEngineApiGetBlobsBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -214,7 +204,7 @@ func (b *consensusEngineApiGetBlobsBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 37) + row := make(map[string]any, 35) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["requested_date_time"] = b.RequestedDateTime.Row(i).UnixMilli() @@ -241,7 +231,6 @@ func (b *consensusEngineApiGetBlobsBatch) Snapshot() []map[string]any { row["meta_execution_version_minor"] = b.MetaExecutionVersionMinor.Row(i) row["meta_execution_version_patch"] = b.MetaExecutionVersionPatch.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -274,7 +263,6 @@ func (b *consensusEngineApiGetBlobsBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/execution/consensus_engine_api_new_payload.gen.go b/pkg/consumoor/route/execution/consensus_engine_api_new_payload.gen.go index aa297d61..32f3c059 100644 --- a/pkg/consumoor/route/execution/consensus_engine_api_new_payload.gen.go +++ b/pkg/consumoor/route/execution/consensus_engine_api_new_payload.gen.go @@ -41,7 +41,6 @@ type consensusEngineApiNewPayloadBatch struct { MetaExecutionVersionMinor proto.ColStr MetaExecutionVersionPatch proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -54,7 +53,6 @@ type consensusEngineApiNewPayloadBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -89,7 +87,6 @@ func (b *consensusEngineApiNewPayloadBatch) appendMetadata(event *xatu.Decorated b.MetaExecutionVersionMinor.Append("") b.MetaExecutionVersionPatch.Append("") b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -102,7 +99,6 @@ func (b *consensusEngineApiNewPayloadBatch) appendMetadata(event *xatu.Decorated b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } @@ -113,7 +109,6 @@ func (b *consensusEngineApiNewPayloadBatch) appendMetadata(event *xatu.Decorated b.MetaExecutionVersionMinor.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionMinor()) b.MetaExecutionVersionPatch.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionPatch()) b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -126,7 +121,6 @@ func (b *consensusEngineApiNewPayloadBatch) appendMetadata(event *xatu.Decorated b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -160,7 +154,6 @@ func (b *consensusEngineApiNewPayloadBatch) Input() proto.Input { {Name: "meta_execution_version_minor", Data: &b.MetaExecutionVersionMinor}, {Name: "meta_execution_version_patch", Data: &b.MetaExecutionVersionPatch}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -173,7 +166,6 @@ func (b *consensusEngineApiNewPayloadBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -207,7 +199,6 @@ func (b *consensusEngineApiNewPayloadBatch) Reset() { b.MetaExecutionVersionMinor.Reset() b.MetaExecutionVersionPatch.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -220,7 +211,6 @@ func (b *consensusEngineApiNewPayloadBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -230,7 +220,7 @@ func (b *consensusEngineApiNewPayloadBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 43) + row := make(map[string]any, 41) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["requested_date_time"] = b.RequestedDateTime.Row(i).UnixMilli() @@ -267,7 +257,6 @@ func (b *consensusEngineApiNewPayloadBatch) Snapshot() []map[string]any { row["meta_execution_version_minor"] = b.MetaExecutionVersionMinor.Row(i) row["meta_execution_version_patch"] = b.MetaExecutionVersionPatch.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -300,7 +289,6 @@ func (b *consensusEngineApiNewPayloadBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/execution/execution_block_metrics.gen.go b/pkg/consumoor/route/execution/execution_block_metrics.gen.go index ac347530..1eaaf8b9 100644 --- a/pkg/consumoor/route/execution/execution_block_metrics.gen.go +++ b/pkg/consumoor/route/execution/execution_block_metrics.gen.go @@ -48,7 +48,6 @@ type executionBlockMetricsBatch struct { CodeCacheHitBytes proto.ColInt64 CodeCacheMissBytes proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -61,9 +60,7 @@ type executionBlockMetricsBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -76,7 +73,6 @@ func newexecutionBlockMetricsBatch() *executionBlockMetricsBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -87,7 +83,6 @@ func (b *executionBlockMetricsBatch) Rows() int { func (b *executionBlockMetricsBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -100,14 +95,11 @@ func (b *executionBlockMetricsBatch) appendMetadata(event *xatu.DecoratedEvent) b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -120,13 +112,7 @@ func (b *executionBlockMetricsBatch) appendMetadata(event *xatu.DecoratedEvent) b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *executionBlockMetricsBatch) Input() proto.Input { @@ -166,7 +152,6 @@ func (b *executionBlockMetricsBatch) Input() proto.Input { {Name: "code_cache_hit_bytes", Data: &b.CodeCacheHitBytes}, {Name: "code_cache_miss_bytes", Data: &b.CodeCacheMissBytes}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -179,9 +164,7 @@ func (b *executionBlockMetricsBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -221,7 +204,6 @@ func (b *executionBlockMetricsBatch) Reset() { b.CodeCacheHitBytes.Reset() b.CodeCacheMissBytes.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -234,9 +216,7 @@ func (b *executionBlockMetricsBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -245,7 +225,7 @@ func (b *executionBlockMetricsBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 51) + row := make(map[string]any, 48) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["source"] = b.Source.Row(i) @@ -281,7 +261,6 @@ func (b *executionBlockMetricsBatch) Snapshot() []map[string]any { row["code_cache_hit_bytes"] = b.CodeCacheHitBytes.Row(i) row["code_cache_miss_bytes"] = b.CodeCacheMissBytes.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -314,9 +293,7 @@ func (b *executionBlockMetricsBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/execution/execution_engine_get_blobs.gen.go b/pkg/consumoor/route/execution/execution_engine_get_blobs.gen.go index 704da51c..12fa6eef 100644 --- a/pkg/consumoor/route/execution/execution_engine_get_blobs.gen.go +++ b/pkg/consumoor/route/execution/execution_engine_get_blobs.gen.go @@ -16,7 +16,7 @@ type executionEngineGetBlobsBatch struct { UpdatedDateTime proto.ColDateTime EventDateTime proto.ColDateTime64 RequestedDateTime proto.ColDateTime64 - DurationMs proto.ColUInt32 + DurationMs proto.ColUInt64 Source proto.ColStr RequestedCount proto.ColUInt32 VersionedHashes *proto.ColArr[[]byte] @@ -31,7 +31,6 @@ type executionEngineGetBlobsBatch struct { MetaExecutionVersionMinor proto.ColStr MetaExecutionVersionPatch proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -44,7 +43,6 @@ type executionEngineGetBlobsBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -80,7 +78,6 @@ func (b *executionEngineGetBlobsBatch) appendMetadata(event *xatu.DecoratedEvent b.MetaExecutionVersionMinor.Append("") b.MetaExecutionVersionPatch.Append("") b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -93,7 +90,6 @@ func (b *executionEngineGetBlobsBatch) appendMetadata(event *xatu.DecoratedEvent b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } @@ -104,7 +100,6 @@ func (b *executionEngineGetBlobsBatch) appendMetadata(event *xatu.DecoratedEvent b.MetaExecutionVersionMinor.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionMinor()) b.MetaExecutionVersionPatch.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionPatch()) b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -117,7 +112,6 @@ func (b *executionEngineGetBlobsBatch) appendMetadata(event *xatu.DecoratedEvent b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -141,7 +135,6 @@ func (b *executionEngineGetBlobsBatch) Input() proto.Input { {Name: "meta_execution_version_minor", Data: &b.MetaExecutionVersionMinor}, {Name: "meta_execution_version_patch", Data: &b.MetaExecutionVersionPatch}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -154,7 +147,6 @@ func (b *executionEngineGetBlobsBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -178,7 +170,6 @@ func (b *executionEngineGetBlobsBatch) Reset() { b.MetaExecutionVersionMinor.Reset() b.MetaExecutionVersionPatch.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -191,7 +182,6 @@ func (b *executionEngineGetBlobsBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -201,7 +191,7 @@ func (b *executionEngineGetBlobsBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 33) + row := make(map[string]any, 31) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["requested_date_time"] = b.RequestedDateTime.Row(i).UnixMilli() @@ -224,7 +214,6 @@ func (b *executionEngineGetBlobsBatch) Snapshot() []map[string]any { row["meta_execution_version_minor"] = b.MetaExecutionVersionMinor.Row(i) row["meta_execution_version_patch"] = b.MetaExecutionVersionPatch.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -257,7 +246,6 @@ func (b *executionEngineGetBlobsBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/execution/execution_engine_get_blobs.go b/pkg/consumoor/route/execution/execution_engine_get_blobs.go index 57b3d94b..f8e1de95 100644 --- a/pkg/consumoor/route/execution/execution_engine_get_blobs.go +++ b/pkg/consumoor/route/execution/execution_engine_get_blobs.go @@ -90,7 +90,7 @@ func (b *executionEngineGetBlobsBatch) appendPayload(event *xatu.DecoratedEvent) } if durationMs := payload.GetDurationMs(); durationMs != nil { - b.DurationMs.Append(uint32(durationMs.GetValue())) //nolint:gosec // duration fits uint32 + b.DurationMs.Append(uint64(durationMs.GetValue())) } else { b.DurationMs.Append(0) } diff --git a/pkg/consumoor/route/execution/execution_engine_new_payload.gen.go b/pkg/consumoor/route/execution/execution_engine_new_payload.gen.go index 3ed09116..fedab8a9 100644 --- a/pkg/consumoor/route/execution/execution_engine_new_payload.gen.go +++ b/pkg/consumoor/route/execution/execution_engine_new_payload.gen.go @@ -16,7 +16,7 @@ type executionEngineNewPayloadBatch struct { UpdatedDateTime proto.ColDateTime EventDateTime proto.ColDateTime64 RequestedDateTime proto.ColDateTime64 - DurationMs proto.ColUInt32 + DurationMs proto.ColUInt64 Source proto.ColStr BlockNumber proto.ColUInt64 BlockHash route.SafeColFixedStr @@ -35,7 +35,6 @@ type executionEngineNewPayloadBatch struct { MetaExecutionVersionMinor proto.ColStr MetaExecutionVersionPatch proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -48,7 +47,6 @@ type executionEngineNewPayloadBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -81,7 +79,6 @@ func (b *executionEngineNewPayloadBatch) appendMetadata(event *xatu.DecoratedEve b.MetaExecutionVersionMinor.Append("") b.MetaExecutionVersionPatch.Append("") b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -94,7 +91,6 @@ func (b *executionEngineNewPayloadBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } @@ -105,7 +101,6 @@ func (b *executionEngineNewPayloadBatch) appendMetadata(event *xatu.DecoratedEve b.MetaExecutionVersionMinor.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionMinor()) b.MetaExecutionVersionPatch.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionPatch()) b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -118,7 +113,6 @@ func (b *executionEngineNewPayloadBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -146,7 +140,6 @@ func (b *executionEngineNewPayloadBatch) Input() proto.Input { {Name: "meta_execution_version_minor", Data: &b.MetaExecutionVersionMinor}, {Name: "meta_execution_version_patch", Data: &b.MetaExecutionVersionPatch}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -159,7 +152,6 @@ func (b *executionEngineNewPayloadBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -187,7 +179,6 @@ func (b *executionEngineNewPayloadBatch) Reset() { b.MetaExecutionVersionMinor.Reset() b.MetaExecutionVersionPatch.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -200,7 +191,6 @@ func (b *executionEngineNewPayloadBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -210,7 +200,7 @@ func (b *executionEngineNewPayloadBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 37) + row := make(map[string]any, 35) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["requested_date_time"] = b.RequestedDateTime.Row(i).UnixMilli() @@ -241,7 +231,6 @@ func (b *executionEngineNewPayloadBatch) Snapshot() []map[string]any { row["meta_execution_version_minor"] = b.MetaExecutionVersionMinor.Row(i) row["meta_execution_version_patch"] = b.MetaExecutionVersionPatch.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -274,7 +263,6 @@ func (b *executionEngineNewPayloadBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/execution/execution_engine_new_payload.go b/pkg/consumoor/route/execution/execution_engine_new_payload.go index 829178e6..83114b0e 100644 --- a/pkg/consumoor/route/execution/execution_engine_new_payload.go +++ b/pkg/consumoor/route/execution/execution_engine_new_payload.go @@ -102,7 +102,7 @@ func (b *executionEngineNewPayloadBatch) appendPayload(event *xatu.DecoratedEven } if durationMs := payload.GetDurationMs(); durationMs != nil { - b.DurationMs.Append(uint32(durationMs.GetValue())) //nolint:gosec // duration fits uint32 + b.DurationMs.Append(uint64(durationMs.GetValue())) } else { b.DurationMs.Append(0) } diff --git a/pkg/consumoor/route/execution/execution_state_size.gen.go b/pkg/consumoor/route/execution/execution_state_size.gen.go index d01f4a53..5eef3ceb 100644 --- a/pkg/consumoor/route/execution/execution_state_size.gen.go +++ b/pkg/consumoor/route/execution/execution_state_size.gen.go @@ -28,7 +28,6 @@ type executionStateSizeBatch struct { StorageTrienodes proto.ColUInt64 StorageTrienodeBytes proto.ColUInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -41,14 +40,12 @@ type executionStateSizeBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaExecutionVersion proto.ColStr MetaExecutionVersionMajor proto.ColStr MetaExecutionVersionMinor proto.ColStr MetaExecutionVersionPatch proto.ColStr MetaExecutionImplementation proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -61,7 +58,6 @@ func newexecutionStateSizeBatch() *executionStateSizeBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -72,7 +68,6 @@ func (b *executionStateSizeBatch) Rows() int { func (b *executionStateSizeBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -85,19 +80,16 @@ func (b *executionStateSizeBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaExecutionVersion.Append("") b.MetaExecutionVersionMajor.Append("") b.MetaExecutionVersionMinor.Append("") b.MetaExecutionVersionPatch.Append("") b.MetaExecutionImplementation.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -110,18 +102,12 @@ func (b *executionStateSizeBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) b.MetaExecutionVersion.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersion()) b.MetaExecutionVersionMajor.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionMajor()) b.MetaExecutionVersionMinor.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionMinor()) b.MetaExecutionVersionPatch.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetVersionPatch()) b.MetaExecutionImplementation.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetImplementation()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *executionStateSizeBatch) Input() proto.Input { @@ -141,7 +127,6 @@ func (b *executionStateSizeBatch) Input() proto.Input { {Name: "storage_trienodes", Data: &b.StorageTrienodes}, {Name: "storage_trienode_bytes", Data: &b.StorageTrienodeBytes}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -154,14 +139,12 @@ func (b *executionStateSizeBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_execution_version", Data: &b.MetaExecutionVersion}, {Name: "meta_execution_version_major", Data: &b.MetaExecutionVersionMajor}, {Name: "meta_execution_version_minor", Data: &b.MetaExecutionVersionMinor}, {Name: "meta_execution_version_patch", Data: &b.MetaExecutionVersionPatch}, {Name: "meta_execution_implementation", Data: &b.MetaExecutionImplementation}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -181,7 +164,6 @@ func (b *executionStateSizeBatch) Reset() { b.StorageTrienodes.Reset() b.StorageTrienodeBytes.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -194,14 +176,12 @@ func (b *executionStateSizeBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaExecutionVersion.Reset() b.MetaExecutionVersionMajor.Reset() b.MetaExecutionVersionMinor.Reset() b.MetaExecutionVersionPatch.Reset() b.MetaExecutionImplementation.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -210,7 +190,7 @@ func (b *executionStateSizeBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 36) + row := make(map[string]any, 33) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["block_number"] = b.BlockNumber.Row(i) @@ -226,7 +206,6 @@ func (b *executionStateSizeBatch) Snapshot() []map[string]any { row["storage_trienodes"] = b.StorageTrienodes.Row(i) row["storage_trienode_bytes"] = b.StorageTrienodeBytes.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -259,14 +238,12 @@ func (b *executionStateSizeBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_execution_version"] = b.MetaExecutionVersion.Row(i) row["meta_execution_version_major"] = b.MetaExecutionVersionMajor.Row(i) row["meta_execution_version_minor"] = b.MetaExecutionVersionMinor.Row(i) row["meta_execution_version_patch"] = b.MetaExecutionVersionPatch.Row(i) row["meta_execution_implementation"] = b.MetaExecutionImplementation.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/execution/mempool_transaction.gen.go b/pkg/consumoor/route/execution/mempool_transaction.gen.go index e4a98230..6b4ca112 100644 --- a/pkg/consumoor/route/execution/mempool_transaction.gen.go +++ b/pkg/consumoor/route/execution/mempool_transaction.gen.go @@ -33,7 +33,6 @@ type mempoolTransactionBatch struct { BlobSidecarsSize *proto.ColNullable[uint32] BlobSidecarsEmptySize *proto.ColNullable[uint32] MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -46,11 +45,9 @@ type mempoolTransactionBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr MetaExecutionForkIDHash proto.ColStr MetaExecutionForkIDNext proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -73,7 +70,6 @@ func newmempoolTransactionBatch() *mempoolTransactionBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -84,7 +80,6 @@ func (b *mempoolTransactionBatch) Rows() int { func (b *mempoolTransactionBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -97,16 +92,13 @@ func (b *mempoolTransactionBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") b.MetaExecutionForkIDHash.Append("") b.MetaExecutionForkIDNext.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -119,15 +111,9 @@ func (b *mempoolTransactionBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) b.MetaExecutionForkIDHash.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetForkId().GetHash()) b.MetaExecutionForkIDNext.Append(event.GetMeta().GetClient().GetEthereum().GetExecution().GetForkId().GetNext()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *mempoolTransactionBatch) Input() proto.Input { @@ -152,7 +138,6 @@ func (b *mempoolTransactionBatch) Input() proto.Input { {Name: "blob_sidecars_size", Data: b.BlobSidecarsSize}, {Name: "blob_sidecars_empty_size", Data: b.BlobSidecarsEmptySize}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -165,11 +150,9 @@ func (b *mempoolTransactionBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, {Name: "meta_execution_fork_id_hash", Data: &b.MetaExecutionForkIDHash}, {Name: "meta_execution_fork_id_next", Data: &b.MetaExecutionForkIDNext}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -194,7 +177,6 @@ func (b *mempoolTransactionBatch) Reset() { b.BlobSidecarsSize.Reset() b.BlobSidecarsEmptySize.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -207,11 +189,9 @@ func (b *mempoolTransactionBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.MetaExecutionForkIDHash.Reset() b.MetaExecutionForkIDNext.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -220,7 +200,7 @@ func (b *mempoolTransactionBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 38) + row := make(map[string]any, 35) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["hash"] = string(b.Hash.Row(i)) @@ -273,7 +253,6 @@ func (b *mempoolTransactionBatch) Snapshot() []map[string]any { row["blob_sidecars_empty_size"] = nil } row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -306,11 +285,9 @@ func (b *mempoolTransactionBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) row["meta_execution_fork_id_hash"] = b.MetaExecutionForkIDHash.Row(i) row["meta_execution_fork_id_next"] = b.MetaExecutionForkIDNext.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_add_peer.gen.go b/pkg/consumoor/route/libp2p/libp2p_add_peer.gen.go index 34a195f5..a32b5870 100644 --- a/pkg/consumoor/route/libp2p/libp2p_add_peer.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_add_peer.gen.go @@ -18,7 +18,6 @@ type libp2pAddPeerBatch struct { PeerIDUniqueKey proto.ColInt64 Protocol proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -31,7 +30,6 @@ type libp2pAddPeerBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -54,7 +52,6 @@ func (b *libp2pAddPeerBatch) Rows() int { func (b *libp2pAddPeerBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -67,13 +64,11 @@ func (b *libp2pAddPeerBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -86,7 +81,6 @@ func (b *libp2pAddPeerBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -97,7 +91,6 @@ func (b *libp2pAddPeerBatch) Input() proto.Input { {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "protocol", Data: &b.Protocol}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -110,7 +103,6 @@ func (b *libp2pAddPeerBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -121,7 +113,6 @@ func (b *libp2pAddPeerBatch) Reset() { b.PeerIDUniqueKey.Reset() b.Protocol.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -134,7 +125,6 @@ func (b *libp2pAddPeerBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -144,13 +134,12 @@ func (b *libp2pAddPeerBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 20) + row := make(map[string]any, 18) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["protocol"] = b.Protocol.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -183,7 +172,6 @@ func (b *libp2pAddPeerBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_connected.gen.go b/pkg/consumoor/route/libp2p/libp2p_connected.gen.go index bcd9bb9f..574ee035 100644 --- a/pkg/consumoor/route/libp2p/libp2p_connected.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_connected.gen.go @@ -18,7 +18,7 @@ type libp2pConnectedBatch struct { RemotePeerIDUniqueKey proto.ColInt64 RemoteProtocol proto.ColStr RemoteTransportProtocol proto.ColStr - RemotePort proto.ColUInt16 + RemotePort *proto.ColNullable[uint16] RemoteIP *proto.ColNullable[proto.IPv6] RemoteGeoCity proto.ColStr RemoteGeoCountry proto.ColStr @@ -38,7 +38,6 @@ type libp2pConnectedBatch struct { Opened proto.ColDateTime Transient proto.ColBool MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -51,7 +50,6 @@ type libp2pConnectedBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -59,6 +57,7 @@ type libp2pConnectedBatch struct { func newlibp2pConnectedBatch() *libp2pConnectedBatch { return &libp2pConnectedBatch{ EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), + RemotePort: new(proto.ColUInt16).Nullable(), RemoteIP: new(proto.ColIPv6).Nullable(), RemoteGeoLongitude: new(proto.ColFloat64).Nullable(), RemoteGeoLatitude: new(proto.ColFloat64).Nullable(), @@ -79,7 +78,6 @@ func (b *libp2pConnectedBatch) Rows() int { func (b *libp2pConnectedBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -92,13 +90,11 @@ func (b *libp2pConnectedBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -111,7 +107,6 @@ func (b *libp2pConnectedBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -122,7 +117,7 @@ func (b *libp2pConnectedBatch) Input() proto.Input { {Name: "remote_peer_id_unique_key", Data: &b.RemotePeerIDUniqueKey}, {Name: "remote_protocol", Data: &b.RemoteProtocol}, {Name: "remote_transport_protocol", Data: &b.RemoteTransportProtocol}, - {Name: "remote_port", Data: &b.RemotePort}, + {Name: "remote_port", Data: b.RemotePort}, {Name: "remote_ip", Data: b.RemoteIP}, {Name: "remote_geo_city", Data: &b.RemoteGeoCity}, {Name: "remote_geo_country", Data: &b.RemoteGeoCountry}, @@ -142,7 +137,6 @@ func (b *libp2pConnectedBatch) Input() proto.Input { {Name: "opened", Data: &b.Opened}, {Name: "transient", Data: &b.Transient}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -155,7 +149,6 @@ func (b *libp2pConnectedBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -186,7 +179,6 @@ func (b *libp2pConnectedBatch) Reset() { b.Opened.Reset() b.Transient.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -199,7 +191,6 @@ func (b *libp2pConnectedBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -209,13 +200,17 @@ func (b *libp2pConnectedBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 40) + row := make(map[string]any, 38) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["remote_peer_id_unique_key"] = b.RemotePeerIDUniqueKey.Row(i) row["remote_protocol"] = b.RemoteProtocol.Row(i) row["remote_transport_protocol"] = b.RemoteTransportProtocol.Row(i) - row["remote_port"] = b.RemotePort.Row(i) + if v := b.RemotePort.Row(i); v.Set { + row["remote_port"] = v.Value + } else { + row["remote_port"] = nil + } if v := b.RemoteIP.Row(i); v.Set { row["remote_ip"] = net.IP(v.Value[:]).String() } else { @@ -255,7 +250,6 @@ func (b *libp2pConnectedBatch) Snapshot() []map[string]any { row["opened"] = b.Opened.Row(i).Unix() row["transient"] = b.Transient.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -288,7 +282,6 @@ func (b *libp2pConnectedBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_connected.go b/pkg/consumoor/route/libp2p/libp2p_connected.go index b9e60c98..e634559c 100644 --- a/pkg/consumoor/route/libp2p/libp2p_connected.go +++ b/pkg/consumoor/route/libp2p/libp2p_connected.go @@ -101,15 +101,15 @@ func (b *libp2pConnectedBatch) appendPayload( b.RemoteTransportProtocol.Append(addr.Transport) if addr.Port > 0 { - b.RemotePort.Append(uint16(addr.Port)) //nolint:gosec // port fits uint16 + b.RemotePort.Append(chProto.NewNullable[uint16](uint16(addr.Port))) //nolint:gosec // port fits uint16 } else { - b.RemotePort.Append(0) + b.RemotePort.Append(chProto.Nullable[uint16]{}) } } else { b.RemoteProtocol.Append("") b.RemoteIP.Append(chProto.Nullable[chProto.IPv6]{}) b.RemoteTransportProtocol.Append("") - b.RemotePort.Append(0) + b.RemotePort.Append(chProto.Nullable[uint16]{}) } // Parse agent version fields. diff --git a/pkg/consumoor/route/libp2p/libp2p_deliver_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_deliver_message.gen.go index 23776db1..2e0f1635 100644 --- a/pkg/consumoor/route/libp2p/libp2p_deliver_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_deliver_message.gen.go @@ -25,7 +25,6 @@ type libp2pDeliverMessageBatch struct { MessageID proto.ColStr MessageSize proto.ColUInt32 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -38,7 +37,6 @@ type libp2pDeliverMessageBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -61,7 +59,6 @@ func (b *libp2pDeliverMessageBatch) Rows() int { func (b *libp2pDeliverMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -74,13 +71,11 @@ func (b *libp2pDeliverMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -93,7 +88,6 @@ func (b *libp2pDeliverMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -111,7 +105,6 @@ func (b *libp2pDeliverMessageBatch) Input() proto.Input { {Name: "message_id", Data: &b.MessageID}, {Name: "message_size", Data: &b.MessageSize}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -124,7 +117,6 @@ func (b *libp2pDeliverMessageBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -142,7 +134,6 @@ func (b *libp2pDeliverMessageBatch) Reset() { b.MessageID.Reset() b.MessageSize.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -155,7 +146,6 @@ func (b *libp2pDeliverMessageBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -165,7 +155,7 @@ func (b *libp2pDeliverMessageBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 27) + row := make(map[string]any, 25) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["topic_layer"] = b.TopicLayer.Row(i) @@ -178,7 +168,6 @@ func (b *libp2pDeliverMessageBatch) Snapshot() []map[string]any { row["message_id"] = b.MessageID.Row(i) row["message_size"] = b.MessageSize.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -211,7 +200,6 @@ func (b *libp2pDeliverMessageBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_disconnected.gen.go b/pkg/consumoor/route/libp2p/libp2p_disconnected.gen.go index c41ee1c6..200c6abd 100644 --- a/pkg/consumoor/route/libp2p/libp2p_disconnected.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_disconnected.gen.go @@ -18,7 +18,7 @@ type libp2pDisconnectedBatch struct { RemotePeerIDUniqueKey proto.ColInt64 RemoteProtocol proto.ColStr RemoteTransportProtocol proto.ColStr - RemotePort proto.ColUInt16 + RemotePort *proto.ColNullable[uint16] RemoteIP *proto.ColNullable[proto.IPv6] RemoteGeoCity proto.ColStr RemoteGeoCountry proto.ColStr @@ -38,7 +38,6 @@ type libp2pDisconnectedBatch struct { Opened proto.ColDateTime Transient proto.ColBool MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -51,7 +50,6 @@ type libp2pDisconnectedBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -59,6 +57,7 @@ type libp2pDisconnectedBatch struct { func newlibp2pDisconnectedBatch() *libp2pDisconnectedBatch { return &libp2pDisconnectedBatch{ EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), + RemotePort: new(proto.ColUInt16).Nullable(), RemoteIP: new(proto.ColIPv6).Nullable(), RemoteGeoLongitude: new(proto.ColFloat64).Nullable(), RemoteGeoLatitude: new(proto.ColFloat64).Nullable(), @@ -79,7 +78,6 @@ func (b *libp2pDisconnectedBatch) Rows() int { func (b *libp2pDisconnectedBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -92,13 +90,11 @@ func (b *libp2pDisconnectedBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -111,7 +107,6 @@ func (b *libp2pDisconnectedBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -122,7 +117,7 @@ func (b *libp2pDisconnectedBatch) Input() proto.Input { {Name: "remote_peer_id_unique_key", Data: &b.RemotePeerIDUniqueKey}, {Name: "remote_protocol", Data: &b.RemoteProtocol}, {Name: "remote_transport_protocol", Data: &b.RemoteTransportProtocol}, - {Name: "remote_port", Data: &b.RemotePort}, + {Name: "remote_port", Data: b.RemotePort}, {Name: "remote_ip", Data: b.RemoteIP}, {Name: "remote_geo_city", Data: &b.RemoteGeoCity}, {Name: "remote_geo_country", Data: &b.RemoteGeoCountry}, @@ -142,7 +137,6 @@ func (b *libp2pDisconnectedBatch) Input() proto.Input { {Name: "opened", Data: &b.Opened}, {Name: "transient", Data: &b.Transient}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -155,7 +149,6 @@ func (b *libp2pDisconnectedBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -186,7 +179,6 @@ func (b *libp2pDisconnectedBatch) Reset() { b.Opened.Reset() b.Transient.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -199,7 +191,6 @@ func (b *libp2pDisconnectedBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -209,13 +200,17 @@ func (b *libp2pDisconnectedBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 40) + row := make(map[string]any, 38) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["remote_peer_id_unique_key"] = b.RemotePeerIDUniqueKey.Row(i) row["remote_protocol"] = b.RemoteProtocol.Row(i) row["remote_transport_protocol"] = b.RemoteTransportProtocol.Row(i) - row["remote_port"] = b.RemotePort.Row(i) + if v := b.RemotePort.Row(i); v.Set { + row["remote_port"] = v.Value + } else { + row["remote_port"] = nil + } if v := b.RemoteIP.Row(i); v.Set { row["remote_ip"] = net.IP(v.Value[:]).String() } else { @@ -255,7 +250,6 @@ func (b *libp2pDisconnectedBatch) Snapshot() []map[string]any { row["opened"] = b.Opened.Row(i).Unix() row["transient"] = b.Transient.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -288,7 +282,6 @@ func (b *libp2pDisconnectedBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_disconnected.go b/pkg/consumoor/route/libp2p/libp2p_disconnected.go index 472ab31f..08cabf67 100644 --- a/pkg/consumoor/route/libp2p/libp2p_disconnected.go +++ b/pkg/consumoor/route/libp2p/libp2p_disconnected.go @@ -101,15 +101,15 @@ func (b *libp2pDisconnectedBatch) appendPayload( b.RemoteTransportProtocol.Append(addr.Transport) if addr.Port > 0 { - b.RemotePort.Append(uint16(addr.Port)) //nolint:gosec // port fits uint16 + b.RemotePort.Append(chProto.NewNullable[uint16](uint16(addr.Port))) //nolint:gosec // port fits uint16 } else { - b.RemotePort.Append(0) + b.RemotePort.Append(chProto.Nullable[uint16]{}) } } else { b.RemoteProtocol.Append("") b.RemoteIP.Append(chProto.Nullable[chProto.IPv6]{}) b.RemoteTransportProtocol.Append("") - b.RemotePort.Append(0) + b.RemotePort.Append(chProto.Nullable[uint16]{}) } // Parse agent version fields. diff --git a/pkg/consumoor/route/libp2p/libp2p_drop_rpc.gen.go b/pkg/consumoor/route/libp2p/libp2p_drop_rpc.gen.go index d85ec967..f866a157 100644 --- a/pkg/consumoor/route/libp2p/libp2p_drop_rpc.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_drop_rpc.gen.go @@ -18,7 +18,6 @@ type libp2pDropRpcBatch struct { EventDateTime proto.ColDateTime64 PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -31,7 +30,6 @@ type libp2pDropRpcBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -54,7 +52,6 @@ func (b *libp2pDropRpcBatch) Rows() int { func (b *libp2pDropRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -67,13 +64,11 @@ func (b *libp2pDropRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -86,7 +81,6 @@ func (b *libp2pDropRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -97,7 +91,6 @@ func (b *libp2pDropRpcBatch) Input() proto.Input { {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -110,7 +103,6 @@ func (b *libp2pDropRpcBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -121,7 +113,6 @@ func (b *libp2pDropRpcBatch) Reset() { b.EventDateTime.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -134,7 +125,6 @@ func (b *libp2pDropRpcBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -144,13 +134,12 @@ func (b *libp2pDropRpcBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 20) + row := make(map[string]any, 18) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -183,7 +172,6 @@ func (b *libp2pDropRpcBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_duplicate_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_duplicate_message.gen.go index 67b60b94..5316d03a 100644 --- a/pkg/consumoor/route/libp2p/libp2p_duplicate_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_duplicate_message.gen.go @@ -25,7 +25,6 @@ type libp2pDuplicateMessageBatch struct { MessageID proto.ColStr MessageSize proto.ColUInt32 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -38,7 +37,6 @@ type libp2pDuplicateMessageBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -61,7 +59,6 @@ func (b *libp2pDuplicateMessageBatch) Rows() int { func (b *libp2pDuplicateMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -74,13 +71,11 @@ func (b *libp2pDuplicateMessageBatch) appendMetadata(event *xatu.DecoratedEvent) b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -93,7 +88,6 @@ func (b *libp2pDuplicateMessageBatch) appendMetadata(event *xatu.DecoratedEvent) b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -111,7 +105,6 @@ func (b *libp2pDuplicateMessageBatch) Input() proto.Input { {Name: "message_id", Data: &b.MessageID}, {Name: "message_size", Data: &b.MessageSize}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -124,7 +117,6 @@ func (b *libp2pDuplicateMessageBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -142,7 +134,6 @@ func (b *libp2pDuplicateMessageBatch) Reset() { b.MessageID.Reset() b.MessageSize.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -155,7 +146,6 @@ func (b *libp2pDuplicateMessageBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -165,7 +155,7 @@ func (b *libp2pDuplicateMessageBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 27) + row := make(map[string]any, 25) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["topic_layer"] = b.TopicLayer.Row(i) @@ -178,7 +168,6 @@ func (b *libp2pDuplicateMessageBatch) Snapshot() []map[string]any { row["message_id"] = b.MessageID.Row(i) row["message_size"] = b.MessageSize.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -211,7 +200,6 @@ func (b *libp2pDuplicateMessageBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.gen.go index 03529a33..5c2f01db 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.gen.go @@ -14,7 +14,6 @@ const libp2pGossipsubAggregateAndProofTableName route.TableName = "libp2p_gossip type libp2pGossipsubAggregateAndProofBatch struct { UpdatedDateTime proto.ColDateTime - Version proto.ColUInt32 EventDateTime proto.ColDateTime64 Slot proto.ColUInt32 SlotStartDateTime proto.ColDateTime @@ -41,7 +40,6 @@ type libp2pGossipsubAggregateAndProofBatch struct { TargetEpoch proto.ColUInt32 TargetRoot route.SafeColFixedStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -54,7 +52,6 @@ type libp2pGossipsubAggregateAndProofBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -80,7 +77,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) Rows() int { func (b *libp2pGossipsubAggregateAndProofBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -93,13 +89,11 @@ func (b *libp2pGossipsubAggregateAndProofBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -112,14 +106,12 @@ func (b *libp2pGossipsubAggregateAndProofBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } func (b *libp2pGossipsubAggregateAndProofBatch) Input() proto.Input { return proto.Input{ {Name: "updated_date_time", Data: &b.UpdatedDateTime}, - {Name: "version", Data: &b.Version}, {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "slot", Data: &b.Slot}, {Name: "slot_start_date_time", Data: &b.SlotStartDateTime}, @@ -146,7 +138,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) Input() proto.Input { {Name: "target_epoch", Data: &b.TargetEpoch}, {Name: "target_root", Data: &b.TargetRoot}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -159,14 +150,12 @@ func (b *libp2pGossipsubAggregateAndProofBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } func (b *libp2pGossipsubAggregateAndProofBatch) Reset() { b.UpdatedDateTime.Reset() - b.Version.Reset() b.EventDateTime.Reset() b.Slot.Reset() b.SlotStartDateTime.Reset() @@ -193,7 +182,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) Reset() { b.TargetEpoch.Reset() b.TargetRoot.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -206,7 +194,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -216,9 +203,8 @@ func (b *libp2pGossipsubAggregateAndProofBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 43) + row := make(map[string]any, 40) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() - row["version"] = b.Version.Row(i) row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -245,7 +231,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) Snapshot() []map[string]any { row["target_epoch"] = b.TargetEpoch.Row(i) row["target_root"] = string(b.TargetRoot.Row(i)) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -278,7 +263,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.go index 02db7e03..4ae096c0 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_aggregate_and_proof.go @@ -187,7 +187,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.AggregatorIndex.Append(0) b.MessageID.Append("") b.MessageSize.Append(0) @@ -211,7 +210,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.AggregatorIndex.Append(0) b.MessageID.Append("") b.MessageSize.Append(0) @@ -225,8 +223,6 @@ func (b *libp2pGossipsubAggregateAndProofBatch) appendClientAdditionalData( } // Extract slot/epoch/wallclock/propagation fields. - var propagationSlotStartDiff uint32 - setGossipsubSlotEpochFields(additional, func(f gossipsubSlotEpochResult) { b.Slot.Append(f.Slot) b.SlotStartDateTime.Append(time.Unix(f.SlotStartDateTime, 0)) @@ -237,12 +233,8 @@ func (b *libp2pGossipsubAggregateAndProofBatch) appendClientAdditionalData( b.WallclockEpoch.Append(f.WallclockEpoch) b.WallclockEpochStartDateTime.Append(time.Unix(f.WallclockEpochStartDateTime, 0)) b.PropagationSlotStartDiff.Append(f.PropagationSlotStartDiff) - propagationSlotStartDiff = f.PropagationSlotStartDiff }) - // Compute version for ReplacingMergeTree dedup. - b.Version.Append(4294967295 - propagationSlotStartDiff) - if aggIdx := additional.GetAggregatorIndex(); aggIdx != nil { b.AggregatorIndex.Append(uint32(aggIdx.GetValue())) } else { diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.gen.go index 365693ed..5e071e89 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.gen.go @@ -14,7 +14,6 @@ const libp2pGossipsubBeaconAttestationTableName route.TableName = "libp2p_gossip type libp2pGossipsubBeaconAttestationBatch struct { UpdatedDateTime proto.ColDateTime - Version proto.ColUInt32 EventDateTime proto.ColDateTime64 Slot proto.ColUInt32 SlotStartDateTime proto.ColDateTime @@ -44,7 +43,6 @@ type libp2pGossipsubBeaconAttestationBatch struct { TargetEpochStartDateTime proto.ColDateTime TargetRoot route.SafeColFixedStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -57,7 +55,6 @@ type libp2pGossipsubBeaconAttestationBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -84,7 +81,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) Rows() int { func (b *libp2pGossipsubBeaconAttestationBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -97,13 +93,11 @@ func (b *libp2pGossipsubBeaconAttestationBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -116,14 +110,12 @@ func (b *libp2pGossipsubBeaconAttestationBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } func (b *libp2pGossipsubBeaconAttestationBatch) Input() proto.Input { return proto.Input{ {Name: "updated_date_time", Data: &b.UpdatedDateTime}, - {Name: "version", Data: &b.Version}, {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "slot", Data: &b.Slot}, {Name: "slot_start_date_time", Data: &b.SlotStartDateTime}, @@ -153,7 +145,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) Input() proto.Input { {Name: "target_epoch_start_date_time", Data: &b.TargetEpochStartDateTime}, {Name: "target_root", Data: &b.TargetRoot}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -166,14 +157,12 @@ func (b *libp2pGossipsubBeaconAttestationBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } func (b *libp2pGossipsubBeaconAttestationBatch) Reset() { b.UpdatedDateTime.Reset() - b.Version.Reset() b.EventDateTime.Reset() b.Slot.Reset() b.SlotStartDateTime.Reset() @@ -203,7 +192,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) Reset() { b.TargetEpochStartDateTime.Reset() b.TargetRoot.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -216,7 +204,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -226,9 +213,8 @@ func (b *libp2pGossipsubBeaconAttestationBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 46) + row := make(map[string]any, 43) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() - row["version"] = b.Version.Row(i) row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -262,7 +248,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) Snapshot() []map[string]any { row["target_epoch_start_date_time"] = b.TargetEpochStartDateTime.Row(i).Unix() row["target_root"] = string(b.TargetRoot.Row(i)) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -295,7 +280,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.go index 27ea397e..67c20456 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_attestation.go @@ -151,7 +151,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.SourceEpochStartDateTime.Append(time.Time{}) b.TargetEpochStartDateTime.Append(time.Time{}) b.AttestingValidatorCommitteeIndex.Append("") @@ -178,7 +177,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.SourceEpochStartDateTime.Append(time.Time{}) b.TargetEpochStartDateTime.Append(time.Time{}) b.AttestingValidatorCommitteeIndex.Append("") @@ -195,8 +193,6 @@ func (b *libp2pGossipsubBeaconAttestationBatch) appendClientAdditionalData( } // Extract slot/epoch/wallclock/propagation fields. - var propagationSlotStartDiff uint32 - setGossipsubSlotEpochFields(additional, func(f gossipsubSlotEpochResult) { b.Slot.Append(f.Slot) b.SlotStartDateTime.Append(time.Unix(f.SlotStartDateTime, 0)) @@ -207,12 +203,8 @@ func (b *libp2pGossipsubBeaconAttestationBatch) appendClientAdditionalData( b.WallclockEpoch.Append(f.WallclockEpoch) b.WallclockEpochStartDateTime.Append(time.Unix(f.WallclockEpochStartDateTime, 0)) b.PropagationSlotStartDiff.Append(f.PropagationSlotStartDiff) - propagationSlotStartDiff = f.PropagationSlotStartDiff }) - // Compute version for ReplacingMergeTree dedup. - b.Version.Append(4294967295 - propagationSlotStartDiff) - // Extract source/target epoch datetime from additional data. if source := additional.GetSource(); source != nil { if epoch := source.GetEpoch(); epoch != nil { diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.gen.go index 74cfd4ba..4d2d3320 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.gen.go @@ -14,7 +14,6 @@ const libp2pGossipsubBeaconBlockTableName route.TableName = "libp2p_gossipsub_be type libp2pGossipsubBeaconBlockBatch struct { UpdatedDateTime proto.ColDateTime - Version proto.ColUInt32 EventDateTime proto.ColDateTime64 Slot proto.ColUInt32 SlotStartDateTime proto.ColDateTime @@ -35,7 +34,6 @@ type libp2pGossipsubBeaconBlockBatch struct { TopicName proto.ColStr TopicEncoding proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -48,7 +46,6 @@ type libp2pGossipsubBeaconBlockBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -72,7 +69,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) Rows() int { func (b *libp2pGossipsubBeaconBlockBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -85,13 +81,11 @@ func (b *libp2pGossipsubBeaconBlockBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -104,14 +98,12 @@ func (b *libp2pGossipsubBeaconBlockBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } func (b *libp2pGossipsubBeaconBlockBatch) Input() proto.Input { return proto.Input{ {Name: "updated_date_time", Data: &b.UpdatedDateTime}, - {Name: "version", Data: &b.Version}, {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "slot", Data: &b.Slot}, {Name: "slot_start_date_time", Data: &b.SlotStartDateTime}, @@ -132,7 +124,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) Input() proto.Input { {Name: "topic_name", Data: &b.TopicName}, {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -145,14 +136,12 @@ func (b *libp2pGossipsubBeaconBlockBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } func (b *libp2pGossipsubBeaconBlockBatch) Reset() { b.UpdatedDateTime.Reset() - b.Version.Reset() b.EventDateTime.Reset() b.Slot.Reset() b.SlotStartDateTime.Reset() @@ -173,7 +162,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) Reset() { b.TopicName.Reset() b.TopicEncoding.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -186,7 +174,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -196,9 +183,8 @@ func (b *libp2pGossipsubBeaconBlockBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 37) + row := make(map[string]any, 34) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() - row["version"] = b.Version.Row(i) row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -219,7 +205,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) Snapshot() []map[string]any { row["topic_name"] = b.TopicName.Row(i) row["topic_encoding"] = b.TopicEncoding.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -252,7 +237,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.go index 89abdd48..37f733d0 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_beacon_block.go @@ -127,7 +127,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.MessageID.Append("") b.MessageSize.Append(0) b.TopicLayer.Append("") @@ -150,7 +149,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.MessageID.Append("") b.MessageSize.Append(0) b.TopicLayer.Append("") @@ -163,8 +161,6 @@ func (b *libp2pGossipsubBeaconBlockBatch) appendClientAdditionalData( } // Extract slot/epoch/wallclock/propagation fields. - var propagationSlotStartDiff uint32 - setGossipsubSlotEpochFields(additional, func(f gossipsubSlotEpochResult) { b.Slot.Append(f.Slot) b.SlotStartDateTime.Append(time.Unix(f.SlotStartDateTime, 0)) @@ -175,12 +171,8 @@ func (b *libp2pGossipsubBeaconBlockBatch) appendClientAdditionalData( b.WallclockEpoch.Append(f.WallclockEpoch) b.WallclockEpochStartDateTime.Append(time.Unix(f.WallclockEpochStartDateTime, 0)) b.PropagationSlotStartDiff.Append(f.PropagationSlotStartDiff) - propagationSlotStartDiff = f.PropagationSlotStartDiff }) - // Compute version for ReplacingMergeTree dedup. - b.Version.Append(4294967295 - propagationSlotStartDiff) - // Extract message fields. b.MessageID.Append(wrappedStringValue(additional.GetMessageId())) diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.gen.go index 315e791d..cad67fa1 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.gen.go @@ -14,7 +14,6 @@ const libp2pGossipsubBlobSidecarTableName route.TableName = "libp2p_gossipsub_bl type libp2pGossipsubBlobSidecarBatch struct { UpdatedDateTime proto.ColDateTime - Version proto.ColUInt32 EventDateTime proto.ColDateTime64 Slot proto.ColUInt32 SlotStartDateTime proto.ColDateTime @@ -38,7 +37,6 @@ type libp2pGossipsubBlobSidecarBatch struct { TopicName proto.ColStr TopicEncoding proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -51,7 +49,6 @@ type libp2pGossipsubBlobSidecarBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -77,7 +74,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) Rows() int { func (b *libp2pGossipsubBlobSidecarBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -90,13 +86,11 @@ func (b *libp2pGossipsubBlobSidecarBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -109,14 +103,12 @@ func (b *libp2pGossipsubBlobSidecarBatch) appendMetadata(event *xatu.DecoratedEv b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } func (b *libp2pGossipsubBlobSidecarBatch) Input() proto.Input { return proto.Input{ {Name: "updated_date_time", Data: &b.UpdatedDateTime}, - {Name: "version", Data: &b.Version}, {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "slot", Data: &b.Slot}, {Name: "slot_start_date_time", Data: &b.SlotStartDateTime}, @@ -140,7 +132,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) Input() proto.Input { {Name: "topic_name", Data: &b.TopicName}, {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -153,14 +144,12 @@ func (b *libp2pGossipsubBlobSidecarBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } func (b *libp2pGossipsubBlobSidecarBatch) Reset() { b.UpdatedDateTime.Reset() - b.Version.Reset() b.EventDateTime.Reset() b.Slot.Reset() b.SlotStartDateTime.Reset() @@ -184,7 +173,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) Reset() { b.TopicName.Reset() b.TopicEncoding.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -197,7 +185,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -207,9 +194,8 @@ func (b *libp2pGossipsubBlobSidecarBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 40) + row := make(map[string]any, 37) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() - row["version"] = b.Version.Row(i) row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -233,7 +219,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) Snapshot() []map[string]any { row["topic_name"] = b.TopicName.Row(i) row["topic_encoding"] = b.TopicEncoding.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -266,7 +251,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.go index 7120e87a..9ea7c25b 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_blob_sidecar.go @@ -139,7 +139,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.MessageID.Append("") b.MessageSize.Append(0) b.TopicLayer.Append("") @@ -162,7 +161,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.MessageID.Append("") b.MessageSize.Append(0) b.TopicLayer.Append("") @@ -175,8 +173,6 @@ func (b *libp2pGossipsubBlobSidecarBatch) appendClientAdditionalData( } // Extract slot/epoch/wallclock/propagation fields. - var propagationSlotStartDiff uint32 - setGossipsubSlotEpochFields(additional, func(f gossipsubSlotEpochResult) { b.Slot.Append(f.Slot) b.SlotStartDateTime.Append(time.Unix(f.SlotStartDateTime, 0)) @@ -187,12 +183,8 @@ func (b *libp2pGossipsubBlobSidecarBatch) appendClientAdditionalData( b.WallclockEpoch.Append(f.WallclockEpoch) b.WallclockEpochStartDateTime.Append(time.Unix(f.WallclockEpochStartDateTime, 0)) b.PropagationSlotStartDiff.Append(f.PropagationSlotStartDiff) - propagationSlotStartDiff = f.PropagationSlotStartDiff }) - // Compute version for ReplacingMergeTree dedup. - b.Version.Append(4294967295 - propagationSlotStartDiff) - // Extract message fields. b.MessageID.Append(wrappedStringValue(additional.GetMessageId())) diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.gen.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.gen.go index d4fab093..07a5a57e 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.gen.go @@ -14,7 +14,6 @@ const libp2pGossipsubDataColumnSidecarTableName route.TableName = "libp2p_gossip type libp2pGossipsubDataColumnSidecarBatch struct { UpdatedDateTime proto.ColDateTime - Version proto.ColUInt32 EventDateTime proto.ColDateTime64 Slot proto.ColUInt32 SlotStartDateTime proto.ColDateTime @@ -39,7 +38,6 @@ type libp2pGossipsubDataColumnSidecarBatch struct { TopicName proto.ColStr TopicEncoding proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -52,7 +50,6 @@ type libp2pGossipsubDataColumnSidecarBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -78,7 +75,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) Rows() int { func (b *libp2pGossipsubDataColumnSidecarBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -91,13 +87,11 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -110,14 +104,12 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } func (b *libp2pGossipsubDataColumnSidecarBatch) Input() proto.Input { return proto.Input{ {Name: "updated_date_time", Data: &b.UpdatedDateTime}, - {Name: "version", Data: &b.Version}, {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "slot", Data: &b.Slot}, {Name: "slot_start_date_time", Data: &b.SlotStartDateTime}, @@ -142,7 +134,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) Input() proto.Input { {Name: "topic_name", Data: &b.TopicName}, {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -155,14 +146,12 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } func (b *libp2pGossipsubDataColumnSidecarBatch) Reset() { b.UpdatedDateTime.Reset() - b.Version.Reset() b.EventDateTime.Reset() b.Slot.Reset() b.SlotStartDateTime.Reset() @@ -187,7 +176,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) Reset() { b.TopicName.Reset() b.TopicEncoding.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -200,7 +188,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -210,9 +197,8 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 41) + row := make(map[string]any, 38) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() - row["version"] = b.Version.Row(i) row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) row["slot_start_date_time"] = b.SlotStartDateTime.Row(i).Unix() @@ -237,7 +223,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) Snapshot() []map[string]any { row["topic_name"] = b.TopicName.Row(i) row["topic_encoding"] = b.TopicEncoding.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -270,7 +255,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.go b/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.go index 228eb8d1..8197823c 100644 --- a/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.go +++ b/pkg/consumoor/route/libp2p/libp2p_gossipsub_data_column_sidecar.go @@ -149,7 +149,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.MessageID.Append("") b.MessageSize.Append(0) b.TopicLayer.Append("") @@ -172,7 +171,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) appendClientAdditionalData( b.WallclockEpoch.Append(0) b.WallclockEpochStartDateTime.Append(time.Time{}) b.PropagationSlotStartDiff.Append(0) - b.Version.Append(4294967295) b.MessageID.Append("") b.MessageSize.Append(0) b.TopicLayer.Append("") @@ -185,8 +183,6 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) appendClientAdditionalData( } // Extract slot/epoch/wallclock/propagation fields. - var propagationSlotStartDiff uint32 - setGossipsubSlotEpochFields(additional, func(f gossipsubSlotEpochResult) { b.Slot.Append(f.Slot) b.SlotStartDateTime.Append(time.Unix(f.SlotStartDateTime, 0)) @@ -197,12 +193,8 @@ func (b *libp2pGossipsubDataColumnSidecarBatch) appendClientAdditionalData( b.WallclockEpoch.Append(f.WallclockEpoch) b.WallclockEpochStartDateTime.Append(time.Unix(f.WallclockEpochStartDateTime, 0)) b.PropagationSlotStartDiff.Append(f.PropagationSlotStartDiff) - propagationSlotStartDiff = f.PropagationSlotStartDiff }) - // Compute version for ReplacingMergeTree dedup. - b.Version.Append(4294967295 - propagationSlotStartDiff) - // Extract message fields. b.MessageID.Append(wrappedStringValue(additional.GetMessageId())) diff --git a/pkg/consumoor/route/libp2p/libp2p_graft.gen.go b/pkg/consumoor/route/libp2p/libp2p_graft.gen.go index f229fdb0..011b6fb0 100644 --- a/pkg/consumoor/route/libp2p/libp2p_graft.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_graft.gen.go @@ -21,7 +21,6 @@ type libp2pGraftBatch struct { TopicEncoding proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -34,7 +33,6 @@ type libp2pGraftBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -57,7 +55,6 @@ func (b *libp2pGraftBatch) Rows() int { func (b *libp2pGraftBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -70,13 +67,11 @@ func (b *libp2pGraftBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -89,7 +84,6 @@ func (b *libp2pGraftBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -103,7 +97,6 @@ func (b *libp2pGraftBatch) Input() proto.Input { {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -116,7 +109,6 @@ func (b *libp2pGraftBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -130,7 +122,6 @@ func (b *libp2pGraftBatch) Reset() { b.TopicEncoding.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -143,7 +134,6 @@ func (b *libp2pGraftBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -153,7 +143,7 @@ func (b *libp2pGraftBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 23) + row := make(map[string]any, 21) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["topic_layer"] = b.TopicLayer.Row(i) @@ -162,7 +152,6 @@ func (b *libp2pGraftBatch) Snapshot() []map[string]any { row["topic_encoding"] = b.TopicEncoding.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -195,7 +184,6 @@ func (b *libp2pGraftBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_handle_metadata.gen.go b/pkg/consumoor/route/libp2p/libp2p_handle_metadata.gen.go index 9da3d1c0..b1c15261 100644 --- a/pkg/consumoor/route/libp2p/libp2p_handle_metadata.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_handle_metadata.gen.go @@ -18,14 +18,13 @@ type libp2pHandleMetadataBatch struct { PeerIDUniqueKey proto.ColInt64 Error *proto.ColNullable[string] Protocol proto.ColStr - Direction *proto.ColNullable[string] + Direction proto.ColStr Attnets proto.ColStr SeqNumber proto.ColUInt64 Syncnets proto.ColStr CustodyGroupCount *proto.ColNullable[uint8] LatencyMilliseconds proto.ColDecimal64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -38,7 +37,6 @@ type libp2pHandleMetadataBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -47,7 +45,6 @@ func newlibp2pHandleMetadataBatch() *libp2pHandleMetadataBatch { return &libp2pHandleMetadataBatch{ EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), Error: new(proto.ColStr).Nullable(), - Direction: new(proto.ColStr).Nullable(), CustodyGroupCount: new(proto.ColUInt8).Nullable(), MetaClientIP: new(proto.ColIPv6).Nullable(), MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), @@ -64,7 +61,6 @@ func (b *libp2pHandleMetadataBatch) Rows() int { func (b *libp2pHandleMetadataBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -77,13 +73,11 @@ func (b *libp2pHandleMetadataBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -96,7 +90,6 @@ func (b *libp2pHandleMetadataBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -107,14 +100,13 @@ func (b *libp2pHandleMetadataBatch) Input() proto.Input { {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "error", Data: b.Error}, {Name: "protocol", Data: &b.Protocol}, - {Name: "direction", Data: b.Direction}, + {Name: "direction", Data: &b.Direction}, {Name: "attnets", Data: &b.Attnets}, {Name: "seq_number", Data: &b.SeqNumber}, {Name: "syncnets", Data: &b.Syncnets}, {Name: "custody_group_count", Data: b.CustodyGroupCount}, {Name: "latency_milliseconds", Data: &route.TypedColInput{ColInput: &b.LatencyMilliseconds, CHType: "Decimal(10, 3)"}}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -127,7 +119,6 @@ func (b *libp2pHandleMetadataBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -145,7 +136,6 @@ func (b *libp2pHandleMetadataBatch) Reset() { b.CustodyGroupCount.Reset() b.LatencyMilliseconds.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -158,7 +148,6 @@ func (b *libp2pHandleMetadataBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -168,7 +157,7 @@ func (b *libp2pHandleMetadataBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 27) + row := make(map[string]any, 25) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) @@ -178,11 +167,7 @@ func (b *libp2pHandleMetadataBatch) Snapshot() []map[string]any { row["error"] = nil } row["protocol"] = b.Protocol.Row(i) - if v := b.Direction.Row(i); v.Set { - row["direction"] = v.Value - } else { - row["direction"] = nil - } + row["direction"] = b.Direction.Row(i) row["attnets"] = b.Attnets.Row(i) row["seq_number"] = b.SeqNumber.Row(i) row["syncnets"] = b.Syncnets.Row(i) @@ -193,7 +178,6 @@ func (b *libp2pHandleMetadataBatch) Snapshot() []map[string]any { } row["latency_milliseconds"] = route.FormatDecimal(int64(b.LatencyMilliseconds.Row(i)), 3) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -226,7 +210,6 @@ func (b *libp2pHandleMetadataBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_handle_metadata.go b/pkg/consumoor/route/libp2p/libp2p_handle_metadata.go index 7d68c600..de720da5 100644 --- a/pkg/consumoor/route/libp2p/libp2p_handle_metadata.go +++ b/pkg/consumoor/route/libp2p/libp2p_handle_metadata.go @@ -73,12 +73,7 @@ func (b *libp2pHandleMetadataBatch) appendPayload( b.Protocol.Append(wrappedStringValue(payload.GetProtocolId())) - // Direction (nullable string). - if dir := wrappedStringValue(payload.GetDirection()); dir != "" { - b.Direction.Append(proto.NewNullable[string](dir)) - } else { - b.Direction.Append(proto.Nullable[string]{}) - } + b.Direction.Append(wrappedStringValue(payload.GetDirection())) // Latency: proto stores seconds as float64; convert to Decimal(10,3) ms. // Truncate (not round) to match Vector's VRL behaviour. diff --git a/pkg/consumoor/route/libp2p/libp2p_handle_status.gen.go b/pkg/consumoor/route/libp2p/libp2p_handle_status.gen.go index ddb782b3..6ec2d690 100644 --- a/pkg/consumoor/route/libp2p/libp2p_handle_status.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_handle_status.gen.go @@ -18,7 +18,7 @@ type libp2pHandleStatusBatch struct { PeerIDUniqueKey proto.ColInt64 Error *proto.ColNullable[string] Protocol proto.ColStr - Direction *proto.ColNullable[string] + Direction proto.ColStr RequestFinalizedEpoch *proto.ColNullable[uint32] RequestFinalizedRoot *proto.ColNullable[string] RequestForkDigest proto.ColStr @@ -33,7 +33,6 @@ type libp2pHandleStatusBatch struct { ResponseEarliestAvailableSlot *proto.ColNullable[uint32] LatencyMilliseconds proto.ColDecimal64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -46,7 +45,6 @@ type libp2pHandleStatusBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -55,7 +53,6 @@ func newlibp2pHandleStatusBatch() *libp2pHandleStatusBatch { return &libp2pHandleStatusBatch{ EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), Error: new(proto.ColStr).Nullable(), - Direction: new(proto.ColStr).Nullable(), RequestFinalizedEpoch: new(proto.ColUInt32).Nullable(), RequestFinalizedRoot: new(proto.ColStr).Nullable(), RequestHeadRoot: route.NewNullableFixedStr(66), @@ -81,7 +78,6 @@ func (b *libp2pHandleStatusBatch) Rows() int { func (b *libp2pHandleStatusBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -94,13 +90,11 @@ func (b *libp2pHandleStatusBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -113,7 +107,6 @@ func (b *libp2pHandleStatusBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -124,7 +117,7 @@ func (b *libp2pHandleStatusBatch) Input() proto.Input { {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "error", Data: b.Error}, {Name: "protocol", Data: &b.Protocol}, - {Name: "direction", Data: b.Direction}, + {Name: "direction", Data: &b.Direction}, {Name: "request_finalized_epoch", Data: b.RequestFinalizedEpoch}, {Name: "request_finalized_root", Data: b.RequestFinalizedRoot}, {Name: "request_fork_digest", Data: &b.RequestForkDigest}, @@ -139,7 +132,6 @@ func (b *libp2pHandleStatusBatch) Input() proto.Input { {Name: "response_earliest_available_slot", Data: b.ResponseEarliestAvailableSlot}, {Name: "latency_milliseconds", Data: &route.TypedColInput{ColInput: &b.LatencyMilliseconds, CHType: "Decimal(10, 3)"}}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -152,7 +144,6 @@ func (b *libp2pHandleStatusBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -178,7 +169,6 @@ func (b *libp2pHandleStatusBatch) Reset() { b.ResponseEarliestAvailableSlot.Reset() b.LatencyMilliseconds.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -191,7 +181,6 @@ func (b *libp2pHandleStatusBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -201,7 +190,7 @@ func (b *libp2pHandleStatusBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 35) + row := make(map[string]any, 33) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) @@ -211,11 +200,7 @@ func (b *libp2pHandleStatusBatch) Snapshot() []map[string]any { row["error"] = nil } row["protocol"] = b.Protocol.Row(i) - if v := b.Direction.Row(i); v.Set { - row["direction"] = v.Value - } else { - row["direction"] = nil - } + row["direction"] = b.Direction.Row(i) if v := b.RequestFinalizedEpoch.Row(i); v.Set { row["request_finalized_epoch"] = v.Value } else { @@ -270,7 +255,6 @@ func (b *libp2pHandleStatusBatch) Snapshot() []map[string]any { } row["latency_milliseconds"] = route.FormatDecimal(int64(b.LatencyMilliseconds.Row(i)), 3) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -303,7 +287,6 @@ func (b *libp2pHandleStatusBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_handle_status.go b/pkg/consumoor/route/libp2p/libp2p_handle_status.go index ec960dd8..6dc825dc 100644 --- a/pkg/consumoor/route/libp2p/libp2p_handle_status.go +++ b/pkg/consumoor/route/libp2p/libp2p_handle_status.go @@ -73,12 +73,7 @@ func (b *libp2pHandleStatusBatch) appendPayload( b.Protocol.Append(wrappedStringValue(payload.GetProtocolId())) - // Direction (nullable string). - if dir := wrappedStringValue(payload.GetDirection()); dir != "" { - b.Direction.Append(proto.NewNullable[string](dir)) - } else { - b.Direction.Append(proto.Nullable[string]{}) - } + b.Direction.Append(wrappedStringValue(payload.GetDirection())) // Latency: proto stores seconds as float64; convert to Decimal(10,3) ms. // Truncate (not round) to match Vector's VRL behaviour. diff --git a/pkg/consumoor/route/libp2p/libp2p_identify.gen.go b/pkg/consumoor/route/libp2p/libp2p_identify.gen.go index 26d1c283..aaef2397 100644 --- a/pkg/consumoor/route/libp2p/libp2p_identify.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_identify.gen.go @@ -46,7 +46,6 @@ type libp2pIdentifyBatch struct { Direction proto.ColStr RemoteMultiaddr proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -59,7 +58,6 @@ type libp2pIdentifyBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -90,7 +88,6 @@ func (b *libp2pIdentifyBatch) Rows() int { func (b *libp2pIdentifyBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -103,13 +100,11 @@ func (b *libp2pIdentifyBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -122,7 +117,6 @@ func (b *libp2pIdentifyBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -161,7 +155,6 @@ func (b *libp2pIdentifyBatch) Input() proto.Input { {Name: "direction", Data: &b.Direction}, {Name: "remote_multiaddr", Data: &b.RemoteMultiaddr}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -174,7 +167,6 @@ func (b *libp2pIdentifyBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -213,7 +205,6 @@ func (b *libp2pIdentifyBatch) Reset() { b.Direction.Reset() b.RemoteMultiaddr.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -226,7 +217,6 @@ func (b *libp2pIdentifyBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -236,7 +226,7 @@ func (b *libp2pIdentifyBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 48) + row := make(map[string]any, 46) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["remote_peer_id_unique_key"] = b.RemotePeerIDUniqueKey.Row(i) @@ -294,7 +284,6 @@ func (b *libp2pIdentifyBatch) Snapshot() []map[string]any { row["direction"] = b.Direction.Row(i) row["remote_multiaddr"] = b.RemoteMultiaddr.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -327,7 +316,6 @@ func (b *libp2pIdentifyBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_join.gen.go b/pkg/consumoor/route/libp2p/libp2p_join.gen.go index 3de919e0..c9cc457e 100644 --- a/pkg/consumoor/route/libp2p/libp2p_join.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_join.gen.go @@ -21,7 +21,6 @@ type libp2pJoinBatch struct { TopicEncoding proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -34,7 +33,6 @@ type libp2pJoinBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -57,7 +55,6 @@ func (b *libp2pJoinBatch) Rows() int { func (b *libp2pJoinBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -70,13 +67,11 @@ func (b *libp2pJoinBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -89,7 +84,6 @@ func (b *libp2pJoinBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -103,7 +97,6 @@ func (b *libp2pJoinBatch) Input() proto.Input { {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -116,7 +109,6 @@ func (b *libp2pJoinBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -130,7 +122,6 @@ func (b *libp2pJoinBatch) Reset() { b.TopicEncoding.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -143,7 +134,6 @@ func (b *libp2pJoinBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -153,7 +143,7 @@ func (b *libp2pJoinBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 23) + row := make(map[string]any, 21) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["topic_layer"] = b.TopicLayer.Row(i) @@ -162,7 +152,6 @@ func (b *libp2pJoinBatch) Snapshot() []map[string]any { row["topic_encoding"] = b.TopicEncoding.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -195,7 +184,6 @@ func (b *libp2pJoinBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_leave.gen.go b/pkg/consumoor/route/libp2p/libp2p_leave.gen.go index 2a3a39b2..0dc08287 100644 --- a/pkg/consumoor/route/libp2p/libp2p_leave.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_leave.gen.go @@ -21,7 +21,6 @@ type libp2pLeaveBatch struct { TopicEncoding proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -34,7 +33,6 @@ type libp2pLeaveBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -57,7 +55,6 @@ func (b *libp2pLeaveBatch) Rows() int { func (b *libp2pLeaveBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -70,13 +67,11 @@ func (b *libp2pLeaveBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -89,7 +84,6 @@ func (b *libp2pLeaveBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -103,7 +97,6 @@ func (b *libp2pLeaveBatch) Input() proto.Input { {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -116,7 +109,6 @@ func (b *libp2pLeaveBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -130,7 +122,6 @@ func (b *libp2pLeaveBatch) Reset() { b.TopicEncoding.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -143,7 +134,6 @@ func (b *libp2pLeaveBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -153,7 +143,7 @@ func (b *libp2pLeaveBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 23) + row := make(map[string]any, 21) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["topic_layer"] = b.TopicLayer.Row(i) @@ -162,7 +152,6 @@ func (b *libp2pLeaveBatch) Snapshot() []map[string]any { row["topic_encoding"] = b.TopicEncoding.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -195,7 +184,6 @@ func (b *libp2pLeaveBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_peer.gen.go b/pkg/consumoor/route/libp2p/libp2p_peer.gen.go index aee18cc7..1648acc8 100644 --- a/pkg/consumoor/route/libp2p/libp2p_peer.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_peer.gen.go @@ -14,7 +14,6 @@ type libp2pPeerBatch struct { UniqueKey proto.ColInt64 UpdatedDateTime proto.ColDateTime PeerID proto.ColStr - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -29,12 +28,10 @@ func (b *libp2pPeerBatch) Rows() int { func (b *libp2pPeerBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -43,7 +40,6 @@ func (b *libp2pPeerBatch) Input() proto.Input { {Name: "unique_key", Data: &b.UniqueKey}, {Name: "updated_date_time", Data: &b.UpdatedDateTime}, {Name: "peer_id", Data: &b.PeerID}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -52,7 +48,6 @@ func (b *libp2pPeerBatch) Reset() { b.UniqueKey.Reset() b.UpdatedDateTime.Reset() b.PeerID.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -62,11 +57,10 @@ func (b *libp2pPeerBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 5) + row := make(map[string]any, 4) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["peer_id"] = b.PeerID.Row(i) - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_prune.gen.go b/pkg/consumoor/route/libp2p/libp2p_prune.gen.go index 30c314d2..5f4ff7b1 100644 --- a/pkg/consumoor/route/libp2p/libp2p_prune.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_prune.gen.go @@ -21,7 +21,6 @@ type libp2pPruneBatch struct { TopicEncoding proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -34,7 +33,6 @@ type libp2pPruneBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -57,7 +55,6 @@ func (b *libp2pPruneBatch) Rows() int { func (b *libp2pPruneBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -70,13 +67,11 @@ func (b *libp2pPruneBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -89,7 +84,6 @@ func (b *libp2pPruneBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -103,7 +97,6 @@ func (b *libp2pPruneBatch) Input() proto.Input { {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -116,7 +109,6 @@ func (b *libp2pPruneBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -130,7 +122,6 @@ func (b *libp2pPruneBatch) Reset() { b.TopicEncoding.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -143,7 +134,6 @@ func (b *libp2pPruneBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -153,7 +143,7 @@ func (b *libp2pPruneBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 23) + row := make(map[string]any, 21) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["topic_layer"] = b.TopicLayer.Row(i) @@ -162,7 +152,6 @@ func (b *libp2pPruneBatch) Snapshot() []map[string]any { row["topic_encoding"] = b.TopicEncoding.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -195,7 +184,6 @@ func (b *libp2pPruneBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_publish_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_publish_message.gen.go index 647bc1a0..0a7d6479 100644 --- a/pkg/consumoor/route/libp2p/libp2p_publish_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_publish_message.gen.go @@ -21,7 +21,6 @@ type libp2pPublishMessageBatch struct { TopicEncoding proto.ColStr MessageID proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -34,7 +33,6 @@ type libp2pPublishMessageBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -57,7 +55,6 @@ func (b *libp2pPublishMessageBatch) Rows() int { func (b *libp2pPublishMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -70,13 +67,11 @@ func (b *libp2pPublishMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -89,7 +84,6 @@ func (b *libp2pPublishMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -103,7 +97,6 @@ func (b *libp2pPublishMessageBatch) Input() proto.Input { {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "message_id", Data: &b.MessageID}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -116,7 +109,6 @@ func (b *libp2pPublishMessageBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -130,7 +122,6 @@ func (b *libp2pPublishMessageBatch) Reset() { b.TopicEncoding.Reset() b.MessageID.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -143,7 +134,6 @@ func (b *libp2pPublishMessageBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -153,7 +143,7 @@ func (b *libp2pPublishMessageBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 23) + row := make(map[string]any, 21) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["topic_layer"] = b.TopicLayer.Row(i) @@ -162,7 +152,6 @@ func (b *libp2pPublishMessageBatch) Snapshot() []map[string]any { row["topic_encoding"] = b.TopicEncoding.Row(i) row["message_id"] = b.MessageID.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -195,7 +184,6 @@ func (b *libp2pPublishMessageBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_recv_rpc.gen.go b/pkg/consumoor/route/libp2p/libp2p_recv_rpc.gen.go index a3857940..0cba9b13 100644 --- a/pkg/consumoor/route/libp2p/libp2p_recv_rpc.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_recv_rpc.gen.go @@ -18,7 +18,6 @@ type libp2pRecvRpcBatch struct { EventDateTime proto.ColDateTime64 PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -31,7 +30,6 @@ type libp2pRecvRpcBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -54,7 +52,6 @@ func (b *libp2pRecvRpcBatch) Rows() int { func (b *libp2pRecvRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -67,13 +64,11 @@ func (b *libp2pRecvRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -86,7 +81,6 @@ func (b *libp2pRecvRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -97,7 +91,6 @@ func (b *libp2pRecvRpcBatch) Input() proto.Input { {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -110,7 +103,6 @@ func (b *libp2pRecvRpcBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -121,7 +113,6 @@ func (b *libp2pRecvRpcBatch) Reset() { b.EventDateTime.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -134,7 +125,6 @@ func (b *libp2pRecvRpcBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -144,13 +134,12 @@ func (b *libp2pRecvRpcBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 20) + row := make(map[string]any, 18) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -183,7 +172,6 @@ func (b *libp2pRecvRpcBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_reject_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_reject_message.gen.go index f82db222..f6f96695 100644 --- a/pkg/consumoor/route/libp2p/libp2p_reject_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_reject_message.gen.go @@ -26,7 +26,6 @@ type libp2pRejectMessageBatch struct { MessageSize proto.ColUInt32 Reason proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -39,7 +38,6 @@ type libp2pRejectMessageBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -62,7 +60,6 @@ func (b *libp2pRejectMessageBatch) Rows() int { func (b *libp2pRejectMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -75,13 +72,11 @@ func (b *libp2pRejectMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -94,7 +89,6 @@ func (b *libp2pRejectMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -113,7 +107,6 @@ func (b *libp2pRejectMessageBatch) Input() proto.Input { {Name: "message_size", Data: &b.MessageSize}, {Name: "reason", Data: &b.Reason}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -126,7 +119,6 @@ func (b *libp2pRejectMessageBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -145,7 +137,6 @@ func (b *libp2pRejectMessageBatch) Reset() { b.MessageSize.Reset() b.Reason.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -158,7 +149,6 @@ func (b *libp2pRejectMessageBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -168,7 +158,7 @@ func (b *libp2pRejectMessageBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 28) + row := make(map[string]any, 26) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["topic_layer"] = b.TopicLayer.Row(i) @@ -182,7 +172,6 @@ func (b *libp2pRejectMessageBatch) Snapshot() []map[string]any { row["message_size"] = b.MessageSize.Row(i) row["reason"] = b.Reason.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -215,7 +204,6 @@ func (b *libp2pRejectMessageBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_remove_peer.gen.go b/pkg/consumoor/route/libp2p/libp2p_remove_peer.gen.go index 40d3885e..018c5c2f 100644 --- a/pkg/consumoor/route/libp2p/libp2p_remove_peer.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_remove_peer.gen.go @@ -17,7 +17,6 @@ type libp2pRemovePeerBatch struct { EventDateTime proto.ColDateTime64 PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -30,7 +29,6 @@ type libp2pRemovePeerBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -53,7 +51,6 @@ func (b *libp2pRemovePeerBatch) Rows() int { func (b *libp2pRemovePeerBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -66,13 +63,11 @@ func (b *libp2pRemovePeerBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -85,7 +80,6 @@ func (b *libp2pRemovePeerBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -95,7 +89,6 @@ func (b *libp2pRemovePeerBatch) Input() proto.Input { {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -108,7 +101,6 @@ func (b *libp2pRemovePeerBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -118,7 +110,6 @@ func (b *libp2pRemovePeerBatch) Reset() { b.EventDateTime.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -131,7 +122,6 @@ func (b *libp2pRemovePeerBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -141,12 +131,11 @@ func (b *libp2pRemovePeerBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 19) + row := make(map[string]any, 17) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -179,7 +168,6 @@ func (b *libp2pRemovePeerBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.gen.go index 86df5320..991c371e 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_data_column_custody_probe.gen.go @@ -31,7 +31,6 @@ type libp2pRpcDataColumnCustodyProbeBatch struct { ResponseTimeMs proto.ColInt32 Error *proto.ColNullable[string] MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -44,9 +43,7 @@ type libp2pRpcDataColumnCustodyProbeBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -60,7 +57,6 @@ func newlibp2pRpcDataColumnCustodyProbeBatch() *libp2pRpcDataColumnCustodyProbeB MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -71,7 +67,6 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) Rows() int { func (b *libp2pRpcDataColumnCustodyProbeBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -84,14 +79,11 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -104,13 +96,7 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) appendMetadata(event *xatu.Decora b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *libp2pRpcDataColumnCustodyProbeBatch) Input() proto.Input { @@ -133,7 +119,6 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) Input() proto.Input { {Name: "response_time_ms", Data: &b.ResponseTimeMs}, {Name: "error", Data: b.Error}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -146,9 +131,7 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -171,7 +154,6 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) Reset() { b.ResponseTimeMs.Reset() b.Error.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -184,9 +166,7 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -195,7 +175,7 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 34) + row := make(map[string]any, 31) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -218,7 +198,6 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) Snapshot() []map[string]any { row["error"] = nil } row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -251,9 +230,7 @@ func (b *libp2pRpcDataColumnCustodyProbeBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_graft.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_graft.gen.go index b4553e62..f35a47ac 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_graft.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_graft.gen.go @@ -24,7 +24,6 @@ type libp2pRpcMetaControlGraftBatch struct { TopicEncoding proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -37,7 +36,6 @@ type libp2pRpcMetaControlGraftBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -60,7 +58,6 @@ func (b *libp2pRpcMetaControlGraftBatch) Rows() int { func (b *libp2pRpcMetaControlGraftBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -73,13 +70,11 @@ func (b *libp2pRpcMetaControlGraftBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -92,7 +87,6 @@ func (b *libp2pRpcMetaControlGraftBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -109,7 +103,6 @@ func (b *libp2pRpcMetaControlGraftBatch) Input() proto.Input { {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -122,7 +115,6 @@ func (b *libp2pRpcMetaControlGraftBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -139,7 +131,6 @@ func (b *libp2pRpcMetaControlGraftBatch) Reset() { b.TopicEncoding.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -152,7 +143,6 @@ func (b *libp2pRpcMetaControlGraftBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -162,7 +152,7 @@ func (b *libp2pRpcMetaControlGraftBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 26) + row := make(map[string]any, 24) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() @@ -174,7 +164,6 @@ func (b *libp2pRpcMetaControlGraftBatch) Snapshot() []map[string]any { row["topic_encoding"] = b.TopicEncoding.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -207,7 +196,6 @@ func (b *libp2pRpcMetaControlGraftBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_idontwant.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_idontwant.gen.go index fb691e45..cf51703e 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_idontwant.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_idontwant.gen.go @@ -22,7 +22,6 @@ type libp2pRpcMetaControlIdontwantBatch struct { MessageID proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -35,7 +34,6 @@ type libp2pRpcMetaControlIdontwantBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -58,7 +56,6 @@ func (b *libp2pRpcMetaControlIdontwantBatch) Rows() int { func (b *libp2pRpcMetaControlIdontwantBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -71,13 +68,11 @@ func (b *libp2pRpcMetaControlIdontwantBatch) appendMetadata(event *xatu.Decorate b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -90,7 +85,6 @@ func (b *libp2pRpcMetaControlIdontwantBatch) appendMetadata(event *xatu.Decorate b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -105,7 +99,6 @@ func (b *libp2pRpcMetaControlIdontwantBatch) Input() proto.Input { {Name: "message_id", Data: &b.MessageID}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -118,7 +111,6 @@ func (b *libp2pRpcMetaControlIdontwantBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -133,7 +125,6 @@ func (b *libp2pRpcMetaControlIdontwantBatch) Reset() { b.MessageID.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -146,7 +137,6 @@ func (b *libp2pRpcMetaControlIdontwantBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -156,7 +146,7 @@ func (b *libp2pRpcMetaControlIdontwantBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 24) + row := make(map[string]any, 22) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() @@ -166,7 +156,6 @@ func (b *libp2pRpcMetaControlIdontwantBatch) Snapshot() []map[string]any { row["message_id"] = b.MessageID.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -199,7 +188,6 @@ func (b *libp2pRpcMetaControlIdontwantBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_ihave.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_ihave.gen.go index b7b4b31a..7c26b463 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_ihave.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_ihave.gen.go @@ -26,7 +26,6 @@ type libp2pRpcMetaControlIhaveBatch struct { MessageID proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -39,7 +38,6 @@ type libp2pRpcMetaControlIhaveBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -62,7 +60,6 @@ func (b *libp2pRpcMetaControlIhaveBatch) Rows() int { func (b *libp2pRpcMetaControlIhaveBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -75,13 +72,11 @@ func (b *libp2pRpcMetaControlIhaveBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -94,7 +89,6 @@ func (b *libp2pRpcMetaControlIhaveBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -113,7 +107,6 @@ func (b *libp2pRpcMetaControlIhaveBatch) Input() proto.Input { {Name: "message_id", Data: &b.MessageID}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -126,7 +119,6 @@ func (b *libp2pRpcMetaControlIhaveBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -145,7 +137,6 @@ func (b *libp2pRpcMetaControlIhaveBatch) Reset() { b.MessageID.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -158,7 +149,6 @@ func (b *libp2pRpcMetaControlIhaveBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -168,7 +158,7 @@ func (b *libp2pRpcMetaControlIhaveBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 28) + row := make(map[string]any, 26) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() @@ -182,7 +172,6 @@ func (b *libp2pRpcMetaControlIhaveBatch) Snapshot() []map[string]any { row["message_id"] = b.MessageID.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -215,7 +204,6 @@ func (b *libp2pRpcMetaControlIhaveBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_iwant.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_iwant.gen.go index 471458b7..74082d56 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_iwant.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_iwant.gen.go @@ -22,7 +22,6 @@ type libp2pRpcMetaControlIwantBatch struct { MessageID proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -35,7 +34,6 @@ type libp2pRpcMetaControlIwantBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -58,7 +56,6 @@ func (b *libp2pRpcMetaControlIwantBatch) Rows() int { func (b *libp2pRpcMetaControlIwantBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -71,13 +68,11 @@ func (b *libp2pRpcMetaControlIwantBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -90,7 +85,6 @@ func (b *libp2pRpcMetaControlIwantBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -105,7 +99,6 @@ func (b *libp2pRpcMetaControlIwantBatch) Input() proto.Input { {Name: "message_id", Data: &b.MessageID}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -118,7 +111,6 @@ func (b *libp2pRpcMetaControlIwantBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -133,7 +125,6 @@ func (b *libp2pRpcMetaControlIwantBatch) Reset() { b.MessageID.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -146,7 +137,6 @@ func (b *libp2pRpcMetaControlIwantBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -156,7 +146,7 @@ func (b *libp2pRpcMetaControlIwantBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 24) + row := make(map[string]any, 22) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() @@ -166,7 +156,6 @@ func (b *libp2pRpcMetaControlIwantBatch) Snapshot() []map[string]any { row["message_id"] = b.MessageID.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -199,7 +188,6 @@ func (b *libp2pRpcMetaControlIwantBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.gen.go index e17fc5e6..67027f66 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_control_prune.gen.go @@ -26,7 +26,6 @@ type libp2pRpcMetaControlPruneBatch struct { TopicName proto.ColStr TopicEncoding proto.ColStr MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -39,7 +38,6 @@ type libp2pRpcMetaControlPruneBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -63,7 +61,6 @@ func (b *libp2pRpcMetaControlPruneBatch) Rows() int { func (b *libp2pRpcMetaControlPruneBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -76,13 +73,11 @@ func (b *libp2pRpcMetaControlPruneBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -95,7 +90,6 @@ func (b *libp2pRpcMetaControlPruneBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -114,7 +108,6 @@ func (b *libp2pRpcMetaControlPruneBatch) Input() proto.Input { {Name: "topic_name", Data: &b.TopicName}, {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -127,7 +120,6 @@ func (b *libp2pRpcMetaControlPruneBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -146,7 +138,6 @@ func (b *libp2pRpcMetaControlPruneBatch) Reset() { b.TopicName.Reset() b.TopicEncoding.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -159,7 +150,6 @@ func (b *libp2pRpcMetaControlPruneBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -169,7 +159,7 @@ func (b *libp2pRpcMetaControlPruneBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 28) + row := make(map[string]any, 26) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() @@ -187,7 +177,6 @@ func (b *libp2pRpcMetaControlPruneBatch) Snapshot() []map[string]any { row["topic_name"] = b.TopicName.Row(i) row["topic_encoding"] = b.TopicEncoding.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -220,7 +209,6 @@ func (b *libp2pRpcMetaControlPruneBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_message.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_message.gen.go index 730b2d27..ee67d9ad 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_message.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_message.gen.go @@ -25,7 +25,6 @@ type libp2pRpcMetaMessageBatch struct { TopicEncoding proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -38,7 +37,6 @@ type libp2pRpcMetaMessageBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -61,7 +59,6 @@ func (b *libp2pRpcMetaMessageBatch) Rows() int { func (b *libp2pRpcMetaMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -74,13 +71,11 @@ func (b *libp2pRpcMetaMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -93,7 +88,6 @@ func (b *libp2pRpcMetaMessageBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -111,7 +105,6 @@ func (b *libp2pRpcMetaMessageBatch) Input() proto.Input { {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -124,7 +117,6 @@ func (b *libp2pRpcMetaMessageBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -142,7 +134,6 @@ func (b *libp2pRpcMetaMessageBatch) Reset() { b.TopicEncoding.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -155,7 +146,6 @@ func (b *libp2pRpcMetaMessageBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -165,7 +155,7 @@ func (b *libp2pRpcMetaMessageBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 27) + row := make(map[string]any, 25) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() @@ -178,7 +168,6 @@ func (b *libp2pRpcMetaMessageBatch) Snapshot() []map[string]any { row["topic_encoding"] = b.TopicEncoding.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -211,7 +200,6 @@ func (b *libp2pRpcMetaMessageBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_subscription.gen.go b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_subscription.gen.go index bf6530dc..3a1c8183 100644 --- a/pkg/consumoor/route/libp2p/libp2p_rpc_meta_subscription.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_rpc_meta_subscription.gen.go @@ -25,7 +25,6 @@ type libp2pRpcMetaSubscriptionBatch struct { TopicEncoding proto.ColStr PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -38,7 +37,6 @@ type libp2pRpcMetaSubscriptionBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -61,7 +59,6 @@ func (b *libp2pRpcMetaSubscriptionBatch) Rows() int { func (b *libp2pRpcMetaSubscriptionBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -74,13 +71,11 @@ func (b *libp2pRpcMetaSubscriptionBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -93,7 +88,6 @@ func (b *libp2pRpcMetaSubscriptionBatch) appendMetadata(event *xatu.DecoratedEve b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -111,7 +105,6 @@ func (b *libp2pRpcMetaSubscriptionBatch) Input() proto.Input { {Name: "topic_encoding", Data: &b.TopicEncoding}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -124,7 +117,6 @@ func (b *libp2pRpcMetaSubscriptionBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -142,7 +134,6 @@ func (b *libp2pRpcMetaSubscriptionBatch) Reset() { b.TopicEncoding.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -155,7 +146,6 @@ func (b *libp2pRpcMetaSubscriptionBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -165,7 +155,7 @@ func (b *libp2pRpcMetaSubscriptionBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 27) + row := make(map[string]any, 25) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() @@ -178,7 +168,6 @@ func (b *libp2pRpcMetaSubscriptionBatch) Snapshot() []map[string]any { row["topic_encoding"] = b.TopicEncoding.Row(i) row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -211,7 +200,6 @@ func (b *libp2pRpcMetaSubscriptionBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_send_rpc.gen.go b/pkg/consumoor/route/libp2p/libp2p_send_rpc.gen.go index e14e7c70..a927d943 100644 --- a/pkg/consumoor/route/libp2p/libp2p_send_rpc.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_send_rpc.gen.go @@ -18,7 +18,6 @@ type libp2pSendRpcBatch struct { EventDateTime proto.ColDateTime64 PeerIDUniqueKey proto.ColInt64 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -31,7 +30,6 @@ type libp2pSendRpcBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -54,7 +52,6 @@ func (b *libp2pSendRpcBatch) Rows() int { func (b *libp2pSendRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -67,13 +64,11 @@ func (b *libp2pSendRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -86,7 +81,6 @@ func (b *libp2pSendRpcBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -97,7 +91,6 @@ func (b *libp2pSendRpcBatch) Input() proto.Input { {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "peer_id_unique_key", Data: &b.PeerIDUniqueKey}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -110,7 +103,6 @@ func (b *libp2pSendRpcBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -121,7 +113,6 @@ func (b *libp2pSendRpcBatch) Reset() { b.EventDateTime.Reset() b.PeerIDUniqueKey.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -134,7 +125,6 @@ func (b *libp2pSendRpcBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -144,13 +134,12 @@ func (b *libp2pSendRpcBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 20) + row := make(map[string]any, 18) row["unique_key"] = b.UniqueKey.Row(i) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["peer_id_unique_key"] = b.PeerIDUniqueKey.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -183,7 +172,6 @@ func (b *libp2pSendRpcBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/libp2p/libp2p_synthetic_heartbeat.gen.go b/pkg/consumoor/route/libp2p/libp2p_synthetic_heartbeat.gen.go index 6d95c199..d80d687a 100644 --- a/pkg/consumoor/route/libp2p/libp2p_synthetic_heartbeat.gen.go +++ b/pkg/consumoor/route/libp2p/libp2p_synthetic_heartbeat.gen.go @@ -38,7 +38,6 @@ type libp2pSyntheticHeartbeatBatch struct { RemoteGeoAutonomousSystemNumber *proto.ColNullable[uint32] RemoteGeoAutonomousSystemOrganization *proto.ColNullable[string] MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -51,7 +50,6 @@ type libp2pSyntheticHeartbeatBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr rows int } @@ -83,7 +81,6 @@ func (b *libp2pSyntheticHeartbeatBatch) Rows() int { func (b *libp2pSyntheticHeartbeatBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -96,13 +93,11 @@ func (b *libp2pSyntheticHeartbeatBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -115,7 +110,6 @@ func (b *libp2pSyntheticHeartbeatBatch) appendMetadata(event *xatu.DecoratedEven b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) } @@ -146,7 +140,6 @@ func (b *libp2pSyntheticHeartbeatBatch) Input() proto.Input { {Name: "remote_geo_autonomous_system_number", Data: b.RemoteGeoAutonomousSystemNumber}, {Name: "remote_geo_autonomous_system_organization", Data: b.RemoteGeoAutonomousSystemOrganization}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -159,7 +152,6 @@ func (b *libp2pSyntheticHeartbeatBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, } } @@ -190,7 +182,6 @@ func (b *libp2pSyntheticHeartbeatBatch) Reset() { b.RemoteGeoAutonomousSystemNumber.Reset() b.RemoteGeoAutonomousSystemOrganization.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -203,7 +194,6 @@ func (b *libp2pSyntheticHeartbeatBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() b.rows = 0 } @@ -213,7 +203,7 @@ func (b *libp2pSyntheticHeartbeatBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 40) + row := make(map[string]any, 38) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["remote_peer_id_unique_key"] = b.RemotePeerIDUniqueKey.Row(i) @@ -271,7 +261,6 @@ func (b *libp2pSyntheticHeartbeatBatch) Snapshot() []map[string]any { row["remote_geo_autonomous_system_organization"] = nil } row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -304,7 +293,6 @@ func (b *libp2pSyntheticHeartbeatBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/mev/mev_relay_bid_trace.gen.go b/pkg/consumoor/route/mev/mev_relay_bid_trace.gen.go index 5cd71214..a9730de2 100644 --- a/pkg/consumoor/route/mev/mev_relay_bid_trace.gen.go +++ b/pkg/consumoor/route/mev/mev_relay_bid_trace.gen.go @@ -40,7 +40,6 @@ type mevRelayBidTraceBatch struct { TimestampMs proto.ColInt64 OptimisticSubmission proto.ColBool MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -54,7 +53,6 @@ type mevRelayBidTraceBatch struct { MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] MetaNetworkName proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -69,7 +67,6 @@ func newmevRelayBidTraceBatch() *mevRelayBidTraceBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -80,7 +77,6 @@ func (b *mevRelayBidTraceBatch) Rows() int { func (b *mevRelayBidTraceBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -94,12 +90,10 @@ func (b *mevRelayBidTraceBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) b.MetaNetworkName.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -113,11 +107,6 @@ func (b *mevRelayBidTraceBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *mevRelayBidTraceBatch) Input() proto.Input { @@ -149,7 +138,6 @@ func (b *mevRelayBidTraceBatch) Input() proto.Input { {Name: "timestamp_ms", Data: &b.TimestampMs}, {Name: "optimistic_submission", Data: &b.OptimisticSubmission}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -163,7 +151,6 @@ func (b *mevRelayBidTraceBatch) Input() proto.Input { {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -195,7 +182,6 @@ func (b *mevRelayBidTraceBatch) Reset() { b.TimestampMs.Reset() b.OptimisticSubmission.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -209,7 +195,6 @@ func (b *mevRelayBidTraceBatch) Reset() { b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() b.MetaNetworkName.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -218,7 +203,7 @@ func (b *mevRelayBidTraceBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 42) + row := make(map[string]any, 40) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -246,7 +231,6 @@ func (b *mevRelayBidTraceBatch) Snapshot() []map[string]any { row["timestamp_ms"] = b.TimestampMs.Row(i) row["optimistic_submission"] = b.OptimisticSubmission.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -280,7 +264,6 @@ func (b *mevRelayBidTraceBatch) Snapshot() []map[string]any { row["meta_client_geo_autonomous_system_organization"] = nil } row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/mev/mev_relay_proposer_payload_delivered.gen.go b/pkg/consumoor/route/mev/mev_relay_proposer_payload_delivered.gen.go index f7a0e560..e5ee4d17 100644 --- a/pkg/consumoor/route/mev/mev_relay_proposer_payload_delivered.gen.go +++ b/pkg/consumoor/route/mev/mev_relay_proposer_payload_delivered.gen.go @@ -34,7 +34,6 @@ type mevRelayProposerPayloadDeliveredBatch struct { Value proto.ColUInt256 NumTx proto.ColUInt32 MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -48,7 +47,6 @@ type mevRelayProposerPayloadDeliveredBatch struct { MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] MetaNetworkName proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -62,7 +60,6 @@ func newmevRelayProposerPayloadDeliveredBatch() *mevRelayProposerPayloadDelivere MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -73,7 +70,6 @@ func (b *mevRelayProposerPayloadDeliveredBatch) Rows() int { func (b *mevRelayProposerPayloadDeliveredBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -87,12 +83,10 @@ func (b *mevRelayProposerPayloadDeliveredBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) b.MetaNetworkName.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -106,11 +100,6 @@ func (b *mevRelayProposerPayloadDeliveredBatch) appendMetadata(event *xatu.Decor b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *mevRelayProposerPayloadDeliveredBatch) Input() proto.Input { @@ -136,7 +125,6 @@ func (b *mevRelayProposerPayloadDeliveredBatch) Input() proto.Input { {Name: "value", Data: &b.Value}, {Name: "num_tx", Data: &b.NumTx}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -150,7 +138,6 @@ func (b *mevRelayProposerPayloadDeliveredBatch) Input() proto.Input { {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -176,7 +163,6 @@ func (b *mevRelayProposerPayloadDeliveredBatch) Reset() { b.Value.Reset() b.NumTx.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -190,7 +176,6 @@ func (b *mevRelayProposerPayloadDeliveredBatch) Reset() { b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() b.MetaNetworkName.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -199,7 +184,7 @@ func (b *mevRelayProposerPayloadDeliveredBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 36) + row := make(map[string]any, 34) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -221,7 +206,6 @@ func (b *mevRelayProposerPayloadDeliveredBatch) Snapshot() []map[string]any { row["value"] = route.UInt256ToString(b.Value.Row(i)) row["num_tx"] = b.NumTx.Row(i) row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -255,7 +239,6 @@ func (b *mevRelayProposerPayloadDeliveredBatch) Snapshot() []map[string]any { row["meta_client_geo_autonomous_system_organization"] = nil } row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/mev/mev_relay_validator_registration.gen.go b/pkg/consumoor/route/mev/mev_relay_validator_registration.gen.go index 48a9c928..8ef1b632 100644 --- a/pkg/consumoor/route/mev/mev_relay_validator_registration.gen.go +++ b/pkg/consumoor/route/mev/mev_relay_validator_registration.gen.go @@ -29,7 +29,6 @@ type mevRelayValidatorRegistrationBatch struct { WallclockEpoch proto.ColUInt32 WallclockEpochStartDateTime proto.ColDateTime MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -43,7 +42,6 @@ type mevRelayValidatorRegistrationBatch struct { MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] MetaNetworkName proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -55,7 +53,6 @@ func newmevRelayValidatorRegistrationBatch() *mevRelayValidatorRegistrationBatch MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -66,7 +63,6 @@ func (b *mevRelayValidatorRegistrationBatch) Rows() int { func (b *mevRelayValidatorRegistrationBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -80,12 +76,10 @@ func (b *mevRelayValidatorRegistrationBatch) appendMetadata(event *xatu.Decorate b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) b.MetaNetworkName.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -99,11 +93,6 @@ func (b *mevRelayValidatorRegistrationBatch) appendMetadata(event *xatu.Decorate b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *mevRelayValidatorRegistrationBatch) Input() proto.Input { @@ -124,7 +113,6 @@ func (b *mevRelayValidatorRegistrationBatch) Input() proto.Input { {Name: "wallclock_epoch", Data: &b.WallclockEpoch}, {Name: "wallclock_epoch_start_date_time", Data: &b.WallclockEpochStartDateTime}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -138,7 +126,6 @@ func (b *mevRelayValidatorRegistrationBatch) Input() proto.Input { {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -159,7 +146,6 @@ func (b *mevRelayValidatorRegistrationBatch) Reset() { b.WallclockEpoch.Reset() b.WallclockEpochStartDateTime.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -173,7 +159,6 @@ func (b *mevRelayValidatorRegistrationBatch) Reset() { b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() b.MetaNetworkName.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -182,7 +167,7 @@ func (b *mevRelayValidatorRegistrationBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 31) + row := make(map[string]any, 29) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["timestamp"] = b.Timestamp.Row(i) @@ -199,7 +184,6 @@ func (b *mevRelayValidatorRegistrationBatch) Snapshot() []map[string]any { row["wallclock_epoch"] = b.WallclockEpoch.Row(i) row["wallclock_epoch_start_date_time"] = b.WallclockEpochStartDateTime.Row(i).Unix() row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -233,7 +217,6 @@ func (b *mevRelayValidatorRegistrationBatch) Snapshot() []map[string]any { row["meta_client_geo_autonomous_system_organization"] = nil } row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/node/node_record_consensus.gen.go b/pkg/consumoor/route/node/node_record_consensus.gen.go index b47b7b68..56cd1a83 100644 --- a/pkg/consumoor/route/node/node_record_consensus.gen.go +++ b/pkg/consumoor/route/node/node_record_consensus.gen.go @@ -17,7 +17,7 @@ type nodeRecordConsensusBatch struct { UpdatedDateTime proto.ColDateTime EventDateTime proto.ColDateTime64 Enr proto.ColStr - NodeID *proto.ColNullable[string] + NodeID proto.ColStr PeerIDUniqueKey *proto.ColNullable[int64] Timestamp proto.ColInt64 Name proto.ColStr @@ -49,7 +49,6 @@ type nodeRecordConsensusBatch struct { GeoAutonomousSystemNumber *proto.ColNullable[uint32] GeoAutonomousSystemOrganization *proto.ColNullable[string] MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -62,16 +61,13 @@ type nodeRecordConsensusBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } func newnodeRecordConsensusBatch() *nodeRecordConsensusBatch { return &nodeRecordConsensusBatch{ EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), - NodeID: new(proto.ColStr).Nullable(), PeerIDUniqueKey: new(proto.ColInt64).Nullable(), NextForkDigest: new(proto.ColStr).Nullable(), Cgc: new(proto.ColStr).Nullable(), @@ -90,7 +86,6 @@ func newnodeRecordConsensusBatch() *nodeRecordConsensusBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -101,7 +96,6 @@ func (b *nodeRecordConsensusBatch) Rows() int { func (b *nodeRecordConsensusBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -114,14 +108,11 @@ func (b *nodeRecordConsensusBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -134,13 +125,7 @@ func (b *nodeRecordConsensusBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *nodeRecordConsensusBatch) Input() proto.Input { @@ -148,7 +133,7 @@ func (b *nodeRecordConsensusBatch) Input() proto.Input { {Name: "updated_date_time", Data: &b.UpdatedDateTime}, {Name: "event_date_time", Data: &b.EventDateTime}, {Name: "enr", Data: &b.Enr}, - {Name: "node_id", Data: b.NodeID}, + {Name: "node_id", Data: &b.NodeID}, {Name: "peer_id_unique_key", Data: b.PeerIDUniqueKey}, {Name: "timestamp", Data: &b.Timestamp}, {Name: "name", Data: &b.Name}, @@ -180,7 +165,6 @@ func (b *nodeRecordConsensusBatch) Input() proto.Input { {Name: "geo_autonomous_system_number", Data: b.GeoAutonomousSystemNumber}, {Name: "geo_autonomous_system_organization", Data: b.GeoAutonomousSystemOrganization}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -193,9 +177,7 @@ func (b *nodeRecordConsensusBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -235,7 +217,6 @@ func (b *nodeRecordConsensusBatch) Reset() { b.GeoAutonomousSystemNumber.Reset() b.GeoAutonomousSystemOrganization.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -248,9 +229,7 @@ func (b *nodeRecordConsensusBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -259,15 +238,11 @@ func (b *nodeRecordConsensusBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 51) + row := make(map[string]any, 48) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["enr"] = b.Enr.Row(i) - if v := b.NodeID.Row(i); v.Set { - row["node_id"] = v.Value - } else { - row["node_id"] = nil - } + row["node_id"] = b.NodeID.Row(i) if v := b.PeerIDUniqueKey.Row(i); v.Set { row["peer_id_unique_key"] = v.Value } else { @@ -351,7 +326,6 @@ func (b *nodeRecordConsensusBatch) Snapshot() []map[string]any { row["geo_autonomous_system_organization"] = nil } row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -384,9 +358,7 @@ func (b *nodeRecordConsensusBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } diff --git a/pkg/consumoor/route/node/node_record_consensus.go b/pkg/consumoor/route/node/node_record_consensus.go index a8d86ba4..257f8884 100644 --- a/pkg/consumoor/route/node/node_record_consensus.go +++ b/pkg/consumoor/route/node/node_record_consensus.go @@ -89,13 +89,7 @@ func (b *nodeRecordConsensusBatch) appendPayload( consensus := event.GetNodeRecordConsensus() b.Enr.Append(nodeStringValue(consensus.GetEnr())) - // NodeID is Nullable[string]. - nodeID := nodeStringValue(consensus.GetNodeId()) - if nodeID != "" { - b.NodeID.Append(chProto.NewNullable[string](nodeID)) - } else { - b.NodeID.Append(chProto.Nullable[string]{}) - } + b.NodeID.Append(nodeStringValue(consensus.GetNodeId())) // PeerIDUniqueKey: hash of peerID + network. peerID := nodeStringValue(consensus.GetPeerId()) diff --git a/pkg/consumoor/route/node/node_record_execution.gen.go b/pkg/consumoor/route/node/node_record_execution.gen.go index eb3101fe..41449b2f 100644 --- a/pkg/consumoor/route/node/node_record_execution.gen.go +++ b/pkg/consumoor/route/node/node_record_execution.gen.go @@ -43,7 +43,6 @@ type nodeRecordExecutionBatch struct { GeoAutonomousSystemNumber *proto.ColNullable[uint32] GeoAutonomousSystemOrganization *proto.ColNullable[string] MetaClientName proto.ColStr - MetaClientID proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr MetaClientOS proto.ColStr @@ -56,9 +55,7 @@ type nodeRecordExecutionBatch struct { MetaClientGeoLatitude *proto.ColNullable[float64] MetaClientGeoAutonomousSystemNumber *proto.ColNullable[uint32] MetaClientGeoAutonomousSystemOrganization *proto.ColNullable[string] - MetaNetworkID proto.ColInt32 MetaNetworkName proto.ColStr - MetaLabels *proto.ColMap[string, string] rows int } @@ -78,7 +75,6 @@ func newnodeRecordExecutionBatch() *nodeRecordExecutionBatch { MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), MetaClientGeoAutonomousSystemNumber: new(proto.ColUInt32).Nullable(), MetaClientGeoAutonomousSystemOrganization: new(proto.ColStr).Nullable(), - MetaLabels: proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), } } @@ -89,7 +85,6 @@ func (b *nodeRecordExecutionBatch) Rows() int { func (b *nodeRecordExecutionBatch) appendMetadata(event *xatu.DecoratedEvent) { if event == nil || event.GetMeta() == nil { b.MetaClientName.Append("") - b.MetaClientID.Append("") b.MetaClientVersion.Append("") b.MetaClientImplementation.Append("") b.MetaClientOS.Append("") @@ -102,14 +97,11 @@ func (b *nodeRecordExecutionBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.Nullable[float64]{}) b.MetaClientGeoAutonomousSystemNumber.Append(proto.Nullable[uint32]{}) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.Nullable[string]{}) - b.MetaNetworkID.Append(0) b.MetaNetworkName.Append("") - b.MetaLabels.Append(nil) return } b.MetaClientName.Append(event.GetMeta().GetClient().GetName()) - b.MetaClientID.Append(event.GetMeta().GetClient().GetId()) b.MetaClientVersion.Append(event.GetMeta().GetClient().GetVersion()) b.MetaClientImplementation.Append(event.GetMeta().GetClient().GetImplementation()) b.MetaClientOS.Append(event.GetMeta().GetClient().GetOs()) @@ -122,13 +114,7 @@ func (b *nodeRecordExecutionBatch) appendMetadata(event *xatu.DecoratedEvent) { b.MetaClientGeoLatitude.Append(proto.NewNullable[float64](event.GetMeta().GetServer().GetClient().GetGeo().GetLatitude())) b.MetaClientGeoAutonomousSystemNumber.Append(proto.NewNullable[uint32](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemNumber())) b.MetaClientGeoAutonomousSystemOrganization.Append(proto.NewNullable[string](event.GetMeta().GetServer().GetClient().GetGeo().GetAutonomousSystemOrganization())) - b.MetaNetworkID.Append(int32(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId())) b.MetaNetworkName.Append(event.GetMeta().GetClient().GetEthereum().GetNetwork().GetName()) - if labels := event.GetMeta().GetClient().GetLabels(); labels != nil { - b.MetaLabels.Append(labels) - } else { - b.MetaLabels.Append(map[string]string{}) - } } func (b *nodeRecordExecutionBatch) Input() proto.Input { @@ -163,7 +149,6 @@ func (b *nodeRecordExecutionBatch) Input() proto.Input { {Name: "geo_autonomous_system_number", Data: b.GeoAutonomousSystemNumber}, {Name: "geo_autonomous_system_organization", Data: b.GeoAutonomousSystemOrganization}, {Name: "meta_client_name", Data: &b.MetaClientName}, - {Name: "meta_client_id", Data: &b.MetaClientID}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, {Name: "meta_client_os", Data: &b.MetaClientOS}, @@ -176,9 +161,7 @@ func (b *nodeRecordExecutionBatch) Input() proto.Input { {Name: "meta_client_geo_latitude", Data: b.MetaClientGeoLatitude}, {Name: "meta_client_geo_autonomous_system_number", Data: b.MetaClientGeoAutonomousSystemNumber}, {Name: "meta_client_geo_autonomous_system_organization", Data: b.MetaClientGeoAutonomousSystemOrganization}, - {Name: "meta_network_id", Data: &b.MetaNetworkID}, {Name: "meta_network_name", Data: &b.MetaNetworkName}, - {Name: "meta_labels", Data: b.MetaLabels}, } } @@ -213,7 +196,6 @@ func (b *nodeRecordExecutionBatch) Reset() { b.GeoAutonomousSystemNumber.Reset() b.GeoAutonomousSystemOrganization.Reset() b.MetaClientName.Reset() - b.MetaClientID.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() b.MetaClientOS.Reset() @@ -226,9 +208,7 @@ func (b *nodeRecordExecutionBatch) Reset() { b.MetaClientGeoLatitude.Reset() b.MetaClientGeoAutonomousSystemNumber.Reset() b.MetaClientGeoAutonomousSystemOrganization.Reset() - b.MetaNetworkID.Reset() b.MetaNetworkName.Reset() - b.MetaLabels.Reset() b.rows = 0 } @@ -237,7 +217,7 @@ func (b *nodeRecordExecutionBatch) Snapshot() []map[string]any { out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 46) + row := make(map[string]any, 43) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["enr"] = b.Enr.Row(i) @@ -296,7 +276,6 @@ func (b *nodeRecordExecutionBatch) Snapshot() []map[string]any { row["geo_autonomous_system_organization"] = nil } row["meta_client_name"] = b.MetaClientName.Row(i) - row["meta_client_id"] = b.MetaClientID.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) row["meta_client_os"] = b.MetaClientOS.Row(i) @@ -329,9 +308,7 @@ func (b *nodeRecordExecutionBatch) Snapshot() []map[string]any { } else { row["meta_client_geo_autonomous_system_organization"] = nil } - row["meta_network_id"] = b.MetaNetworkID.Row(i) row["meta_network_name"] = b.MetaNetworkName.Row(i) - row["meta_labels"] = b.MetaLabels.Row(i) out[i] = row } From f45bf97391b21beeca17f6314b6f00c4871bb467 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 1 Apr 2026 11:12:22 +1000 Subject: [PATCH 22/27] =?UTF-8?q?consumoor:=20fix=20review=20findings=20?= =?UTF-8?q?=E2=80=94=20config=20strictness,=20failure=20isolation,=20watch?= =?UTF-8?q?er=20bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Fix CLI/env precedence: flag now correctly overrides env var - Fix watcher initial discovery: seed from activeTopics so topics appearing between New() and first poll aren't missed - Strict YAML config: unknown keys now cause startup error - Per-table failure isolation: WriteBatch groups by table not event type, so one table's failure can't NAK another table's messages - Allow zero-topic startup when topicRefreshInterval > 0 - Remove dead ConsumoorBufferNearCapacity alert (referenced non-existent metric) - Document fail-fast errgroup policy and concurrency model --- cmd/consumoor.go | 15 ++++-- deploy/prometheus/consumoor.rules.yml | 12 ----- pkg/consumoor/clickhouse/writer.go | 7 +++ pkg/consumoor/consumoor.go | 56 ++++++++++++++-------- pkg/consumoor/source/output.go | 69 +++++++++++---------------- 5 files changed, 81 insertions(+), 78 deletions(-) diff --git a/cmd/consumoor.go b/cmd/consumoor.go index 3379592c..0c914f11 100644 --- a/cmd/consumoor.go +++ b/cmd/consumoor.go @@ -2,6 +2,7 @@ package cmd import ( + "bytes" "os" "github.com/creasty/defaults" @@ -36,14 +37,15 @@ func createConsumoorOverride(config ConsumoorOverrideConfig) ConsumoorOverride { Setter: func(cmd *cobra.Command, overrides *consumoor.Override) error { val := "" - if cmd.Flags().Changed(config.FlagName) { - val = cmd.Flags().Lookup(config.FlagName).Value.String() - } - + // Precedence: flag > env > config file. if os.Getenv(config.EnvName) != "" { val = os.Getenv(config.EnvName) } + if cmd.Flags().Changed(config.FlagName) { + val = cmd.Flags().Lookup(config.FlagName).Value.String() + } + if val == "" { return nil } @@ -134,7 +136,10 @@ func loadConsumoorConfigFromFile(file string) (*consumoor.Config, error) { type plain consumoor.Config - if err := yaml.Unmarshal(yamlFile, (*plain)(config)); err != nil { + dec := yaml.NewDecoder(bytes.NewReader(yamlFile)) + dec.KnownFields(true) + + if err := dec.Decode((*plain)(config)); err != nil { return nil, err } diff --git a/deploy/prometheus/consumoor.rules.yml b/deploy/prometheus/consumoor.rules.yml index 0114d5e9..ec5e1a96 100644 --- a/deploy/prometheus/consumoor.rules.yml +++ b/deploy/prometheus/consumoor.rules.yml @@ -1,18 +1,6 @@ groups: - name: consumoor rules: - - alert: ConsumoorBufferNearCapacity - expr: xatu_consumoor_buffer_usage > 160000 - for: 5m - labels: - severity: warning - annotations: - summary: "Consumoor buffer near capacity for table {{ $labels.table }}" - description: >- - Buffer usage for table {{ $labels.table }} is {{ $value }} rows, - exceeding 80% of the default 200k capacity. The pipeline may start - dropping or blocking messages if the buffer fills up. - - alert: ConsumoorWriteErrorsHigh expr: rate(xatu_consumoor_write_errors_total[5m]) > 0 for: 5m diff --git a/pkg/consumoor/clickhouse/writer.go b/pkg/consumoor/clickhouse/writer.go index 3685f7cb..8eef946c 100644 --- a/pkg/consumoor/clickhouse/writer.go +++ b/pkg/consumoor/clickhouse/writer.go @@ -214,6 +214,13 @@ func (w *ChGoWriter) Stop(_ context.Context) error { // ClickHouse tables concurrently. The map keys are base table names // (without suffix). Returns a FlushResult containing per-table errors // and any invalid events that should be sent to the DLQ. +// +// Concurrency model: each table flush runs in its own goroutine. The +// effective global concurrency cap is the ch-go pool's MaxConns, since +// each flush must acquire a pool connection. The per-table adaptive +// limiter further constrains concurrency per table. Goroutines blocked +// on pool.Do hold their batch data in memory; the aggregate footprint +// is bounded by (streams × maxInFlight × avg_batch_size). func (w *ChGoWriter) FlushTableEvents( ctx context.Context, tableEvents map[string][]*xatu.DecoratedEvent, diff --git a/pkg/consumoor/consumoor.go b/pkg/consumoor/consumoor.go index 4eb2a1fd..9b9f168b 100644 --- a/pkg/consumoor/consumoor.go +++ b/pkg/consumoor/consumoor.go @@ -5,6 +5,7 @@ import ( "encoding/json" "errors" "fmt" + "maps" "net" "net/http" "os/signal" @@ -106,18 +107,23 @@ func New( return nil, fmt.Errorf("discovering kafka topics: %w", err) } + cLog := log.WithField("component", "consumoor") + if len(topics) == 0 { - return nil, fmt.Errorf( - "no kafka topics matched patterns %v", - config.Kafka.Topics, - ) + if config.Kafka.TopicRefreshInterval > 0 { + cLog.Warn("No topics matched at startup — topic watcher will discover them dynamically") + } else { + return nil, fmt.Errorf( + "no kafka topics matched patterns %v", + config.Kafka.Topics, + ) + } + } else { + cLog.WithField("topics", topics). + WithField("count", len(topics)). + Info("Discovered Kafka topics for per-topic streams") } - cLog := log.WithField("component", "consumoor") - cLog.WithField("topics", topics). - WithField("count", len(topics)). - Info("Discovered Kafka topics for per-topic streams") - streams := make([]topicStream, 0, len(topics)) consumerGroups := make([]string, 0, len(topics)) @@ -193,8 +199,10 @@ func New( activeTopics: activeTopics, } - // Optionally create the Kafka consumer lag monitor. - if config.Kafka.LagPollInterval > 0 { + // Optionally create the Kafka consumer lag monitor. Skipped when no + // topics were discovered at startup (dynamic discovery will add them + // later, but lag monitoring requires a restart to include them). + if config.Kafka.LagPollInterval > 0 && len(consumerGroups) > 0 { lagMon, lagErr := source.NewLagMonitor( log, &config.Kafka, @@ -224,6 +232,15 @@ func (c *Consumoor) Start(ctx context.Context) error { c.log.WithField("streams", len(c.streams)). Info("Consumoor started (per-topic benthos streams)") + // errgroup provides fail-fast: if any goroutine (stream, metrics + // server, lag monitor, topic watcher) returns an error, gCtx is + // cancelled and all other goroutines shut down. This is intentional + // — a persistent stream failure typically indicates a systemic + // problem that affects all streams. The orchestrator (k8s, systemd) + // can then restart the entire process. + // + // Implication for dynamic topic discovery: a newly discovered topic + // whose stream fails will bring down all existing healthy streams. g, gCtx := errgroup.WithContext(nctx) g.Go(func() error { @@ -395,8 +412,13 @@ func (c *Consumoor) watchTopics(ctx context.Context, g *errgroup.Group) { c.log.WithField("interval", interval). Info("Starting topic discovery watcher") - // Perform an initial discovery so the metric is populated immediately. - knownTopics := c.discoverAndDiff(ctx, g, nil) + // Seed from activeTopics so topics appearing between New() and the + // first poll are detected as new and get streams started. + c.mu.Lock() + initialKnown := maps.Clone(c.activeTopics) + c.mu.Unlock() + + knownTopics := c.discoverAndDiff(ctx, g, initialKnown) ticker := time.NewTicker(interval) defer ticker.Stop() @@ -435,14 +457,6 @@ func (c *Consumoor) discoverAndDiff( c.metrics.ActiveTopics().Set(float64(len(current))) - if previous == nil { - c.log.WithField("topics", topics). - WithField("count", len(topics)). - Info("Initial topic discovery complete") - - return current - } - // Find newly appeared topics and start streams for them. for _, t := range topics { if _, ok := previous[t]; !ok { diff --git a/pkg/consumoor/source/output.go b/pkg/consumoor/source/output.go index 0587809a..a561754f 100644 --- a/pkg/consumoor/source/output.go +++ b/pkg/consumoor/source/output.go @@ -113,9 +113,13 @@ func (o *xatuClickHouseOutput) WriteBatch( } }() - groups := make(map[xatu.Event_Name]*eventGroup, 16) + // Decode, route, and group by target table in a single pass. Each + // table group is processed independently so a write failure for table + // A cannot NAK messages destined only for table B. Messages that fan + // out to multiple tables appear in multiple groups — each is flushed + // and error-attributed independently. + tableGroups := make(map[string]*eventGroup, 16) - // Phase 1: decode, route, and group by event type. for i, msg := range msgs { kafka := kafkaMetadata(msg) o.metrics.MessagesConsumed().WithLabelValues(kafka.Topic).Inc() @@ -204,40 +208,25 @@ func (o *xatuClickHouseOutput) WriteBatch( continue } - if len(outcome.Results) == 0 { - continue - } - - tables := make([]string, len(outcome.Results)) - for j, result := range outcome.Results { - tables[j] = result.Table - } - - eventName := event.GetEvent().GetName() - - g, ok := groups[eventName] - if !ok { - g = &eventGroup{ - messages: make([]groupMessage, 0, 8), + for _, result := range outcome.Results { + tg, ok := tableGroups[result.Table] + if !ok { + tg = &eventGroup{messages: make([]groupMessage, 0, 8)} + tableGroups[result.Table] = tg } - groups[eventName] = g - } - g.messages = append(g.messages, groupMessage{ - batchIndex: i, - raw: raw, - event: event, - kafka: kafka, - tables: tables, - }) + tg.messages = append(tg.messages, groupMessage{ + batchIndex: i, + raw: raw, + event: event, + kafka: kafka, + tables: []string{result.Table}, + }) + } } - // Phase 2: process each event group independently. - // Pass batchErr through so Phase 1 failures (decode errors) are preserved - // when a group also fails — otherwise processGroup would create a new - // BatchError that silently drops the earlier failures. - for _, g := range groups { - batchErr = o.processGroup(ctx, msgs, batchErr, g) + for _, tg := range tableGroups { + batchErr = o.processGroup(ctx, msgs, batchErr, tg) } if batchErr != nil { @@ -276,15 +265,15 @@ func (o *xatuClickHouseOutput) Close(ctx context.Context) error { return rejectErr } -// processGroup writes all messages in the group to their target tables, then -// flushes only those tables. On failure the entire group is NAK'd or DLQ'd. -// The caller's accumulated batchErr is threaded through so that failures from -// earlier phases (e.g. decode errors) are preserved. +// processGroup writes all messages in the group to their target table and +// handles error attribution. Each group targets exactly one table (callers +// split fanout events into per-table groups), so failure in one table +// cannot affect messages destined for a different table. +// +// The caller's accumulated batchErr is threaded through so that failures +// from earlier phases (e.g. decode errors) are preserved. // -// For events that fan out to multiple tables, partial failures (some tables -// succeed, others fail transiently) are retried with exponential backoff. -// Only the failed tables are retried to prevent duplicate writes on already- -// succeeded tables. +// Transient failures are retried with exponential backoff. func (o *xatuClickHouseOutput) processGroup( ctx context.Context, msgs service.MessageBatch, From e8ad1558824be5272ad26345fa4c43e847ead22d Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 1 Apr 2026 13:03:10 +1000 Subject: [PATCH 23/27] consumoor: fix lint issues from CI - Preallocate pooledEvents slice, fix wsl whitespace - Suppress nilnil on intentional nil reject sink returns - Name return values on LogSampler.Allow (gocritic) - Remove unnecessary uint64 conversion after schema migration changed DurationMs columns from UInt32 to UInt64 - Add blank line before range loop (wsl) --- pkg/consumoor/consumoor.go | 1 + .../beacon/beacon_api_eth_v1_events_data_column_sidecar.go | 1 + pkg/consumoor/route/execution/execution_engine_get_blobs.go | 2 +- pkg/consumoor/route/execution/execution_engine_new_payload.go | 2 +- pkg/consumoor/source/dlq.go | 4 ++-- pkg/consumoor/source/output.go | 3 ++- pkg/consumoor/telemetry/log_sampler.go | 4 ++-- 7 files changed, 10 insertions(+), 7 deletions(-) diff --git a/pkg/consumoor/consumoor.go b/pkg/consumoor/consumoor.go index 9b9f168b..9effd194 100644 --- a/pkg/consumoor/consumoor.go +++ b/pkg/consumoor/consumoor.go @@ -486,6 +486,7 @@ func (c *Consumoor) startTopicStream( topic string, ) { c.mu.Lock() + if _, exists := c.activeTopics[topic]; exists { c.mu.Unlock() diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go index f60b6115..5b68ee3f 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go @@ -107,6 +107,7 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendPayload(event *xatu.D commitments := sidecar.GetKzgCommitments() //nolint:staticcheck // deprecated but still populated byteSlices := make([][]byte, len(commitments)) + for i, c := range commitments { byteSlices[i] = []byte(c) } diff --git a/pkg/consumoor/route/execution/execution_engine_get_blobs.go b/pkg/consumoor/route/execution/execution_engine_get_blobs.go index f8e1de95..e7615d26 100644 --- a/pkg/consumoor/route/execution/execution_engine_get_blobs.go +++ b/pkg/consumoor/route/execution/execution_engine_get_blobs.go @@ -90,7 +90,7 @@ func (b *executionEngineGetBlobsBatch) appendPayload(event *xatu.DecoratedEvent) } if durationMs := payload.GetDurationMs(); durationMs != nil { - b.DurationMs.Append(uint64(durationMs.GetValue())) + b.DurationMs.Append(durationMs.GetValue()) } else { b.DurationMs.Append(0) } diff --git a/pkg/consumoor/route/execution/execution_engine_new_payload.go b/pkg/consumoor/route/execution/execution_engine_new_payload.go index 83114b0e..56bdc48a 100644 --- a/pkg/consumoor/route/execution/execution_engine_new_payload.go +++ b/pkg/consumoor/route/execution/execution_engine_new_payload.go @@ -102,7 +102,7 @@ func (b *executionEngineNewPayloadBatch) appendPayload(event *xatu.DecoratedEven } if durationMs := payload.GetDurationMs(); durationMs != nil { - b.DurationMs.Append(uint64(durationMs.GetValue())) + b.DurationMs.Append(durationMs.GetValue()) } else { b.DurationMs.Append(0) } diff --git a/pkg/consumoor/source/dlq.go b/pkg/consumoor/source/dlq.go index 9d28623b..006ecd0c 100644 --- a/pkg/consumoor/source/dlq.go +++ b/pkg/consumoor/source/dlq.go @@ -88,11 +88,11 @@ func (s *kafkaRejectSink) Write(ctx context.Context, record *rejectedRecord) err func newRejectSink(cfg *KafkaConfig) (rejectSink, error) { if cfg == nil { - return nil, nil + return nil, nil //nolint:nilnil // nil sink is the intentional no-op case } if strings.TrimSpace(cfg.RejectedTopic) == "" { - return nil, nil + return nil, nil //nolint:nilnil // nil sink is the intentional no-op case } opts := []kgo.Opt{ diff --git a/pkg/consumoor/source/output.go b/pkg/consumoor/source/output.go index a561754f..0eec5272 100644 --- a/pkg/consumoor/source/output.go +++ b/pkg/consumoor/source/output.go @@ -106,7 +106,8 @@ func (o *xatuClickHouseOutput) WriteBatch( var batchErr *service.BatchError - var pooledEvents []*xatu.DecoratedEvent + pooledEvents := make([]*xatu.DecoratedEvent, 0, len(msgs)) + defer func() { for _, ev := range pooledEvents { ev.ReturnToVTPool() diff --git a/pkg/consumoor/telemetry/log_sampler.go b/pkg/consumoor/telemetry/log_sampler.go index c721078a..54562841 100644 --- a/pkg/consumoor/telemetry/log_sampler.go +++ b/pkg/consumoor/telemetry/log_sampler.go @@ -33,7 +33,7 @@ func NewLogSampler(interval time.Duration) *LogSampler { // Allow returns whether a log should be emitted for the given key, and // the number of occurrences suppressed since the last emission. -func (s *LogSampler) Allow(key string) (bool, int64) { +func (s *LogSampler) Allow(key string) (allowed bool, suppressed int64) { now := time.Now() s.mu.Lock() @@ -52,7 +52,7 @@ func (s *LogSampler) Allow(key string) (bool, int64) { return false, 0 } - suppressed := entry.suppressed + suppressed = entry.suppressed entry.suppressed = 0 entry.lastEmit = now From a9184a593c028f1cece46a95e092ddf06d8f5e88 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 1 Apr 2026 13:06:50 +1000 Subject: [PATCH 24/27] fix: bump grpc to v1.79.3 for GO-2026-4762, fix buf action hash - google.golang.org/grpc v1.78.0 -> v1.79.3 (authorization bypass vuln) - Fix bufbuild/buf-setup-action pinned hash for v1.50.0 --- .github/workflows/buf.yaml | 2 +- go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/buf.yaml b/.github/workflows/buf.yaml index 8813aab2..74cccc7c 100644 --- a/.github/workflows/buf.yaml +++ b/.github/workflows/buf.yaml @@ -24,7 +24,7 @@ jobs: uses: actions/checkout@08c6903cd8c0fde910a37f88322edcfb5dd907a8 # v5.0.0 - name: Set up buf - uses: bufbuild/buf-setup-action@a47c93e0b1648769eb9a2e1f98e7b4e3e13089f0 # v1.50.0 + uses: bufbuild/buf-setup-action@a47c93e0b1648d5651a065437926377d060baa99 # v1.50.0 - name: Lint protos run: buf lint diff --git a/go.mod b/go.mod index eb44716e..b795616d 100644 --- a/go.mod +++ b/go.mod @@ -63,7 +63,7 @@ require ( golang.org/x/crypto v0.47.0 golang.org/x/sync v0.19.0 golang.org/x/time v0.13.0 - google.golang.org/grpc v1.78.0 + google.golang.org/grpc v1.79.3 google.golang.org/protobuf v1.36.11 gopkg.in/yaml.v3 v3.0.1 ) diff --git a/go.sum b/go.sum index 45ce7e56..5bf023c9 100644 --- a/go.sum +++ b/go.sum @@ -1007,8 +1007,8 @@ google.golang.org/genproto/googleapis/api v0.0.0-20260128011058-8636f8732409 h1: google.golang.org/genproto/googleapis/api v0.0.0-20260128011058-8636f8732409/go.mod h1:fl8J1IvUjCilwZzQowmw2b7HQB2eAuYBabMXzWurF+I= google.golang.org/genproto/googleapis/rpc v0.0.0-20260128011058-8636f8732409 h1:H86B94AW+VfJWDqFeEbBPhEtHzJwJfTbgE2lZa54ZAQ= google.golang.org/genproto/googleapis/rpc v0.0.0-20260128011058-8636f8732409/go.mod h1:j9x/tPzZkyxcgEFkiKEEGxfvyumM01BEtsW8xzOahRQ= -google.golang.org/grpc v1.78.0 h1:K1XZG/yGDJnzMdd/uZHAkVqJE+xIDOcmdSFZkBUicNc= -google.golang.org/grpc v1.78.0/go.mod h1:I47qjTo4OKbMkjA/aOOwxDIiPSBofUtQUI5EfpWvW7U= +google.golang.org/grpc v1.79.3 h1:sybAEdRIEtvcD68Gx7dmnwjZKlyfuc61Dyo9pGXXkKE= +google.golang.org/grpc v1.79.3/go.mod h1:KmT0Kjez+0dde/v2j9vzwoAScgEPx/Bw1CYChhHLrHQ= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= From 902beb96dbf5656feb97f2f867870a502549cf4f Mon Sep 17 00:00:00 2001 From: Andrew Davis <1709934+Savid@users.noreply.github.com> Date: Wed, 1 Apr 2026 17:51:44 +1000 Subject: [PATCH 25/27] fix attestation order by --- deploy/migrations/clickhouse/001_init.up.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/deploy/migrations/clickhouse/001_init.up.sql b/deploy/migrations/clickhouse/001_init.up.sql index 4f8090f5..ea57c032 100644 --- a/deploy/migrations/clickhouse/001_init.up.sql +++ b/deploy/migrations/clickhouse/001_init.up.sql @@ -126,7 +126,8 @@ CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_attestation_local ON ) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{installation}/{cluster}/tables/{shard}/{database}/{table}', '{replica}', updated_date_time) PARTITION BY (meta_network_name, toYYYYMM(slot_start_date_time)) -ORDER BY (meta_network_name, slot_start_date_time, meta_client_name) +ORDER BY (meta_network_name, slot_start_date_time, meta_client_name, attesting_validator_index, attesting_validator_committee_index, aggregation_bits, beacon_block_root) +SETTINGS allow_nullable_key = 1 COMMENT 'Contains beacon API attestation events from each sentry client attached to a beacon node'; CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_blob_sidecar_local ON CLUSTER '{cluster}' From fbf30c6605adaea30e0b09332b96d37903ef9255 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 1 Apr 2026 18:15:40 +1000 Subject: [PATCH 26/27] consumoor: add clientId config for shared Kafka client-id All per-topic benthos streams now optionally share a single Kafka client-id when clientId is set, enabling broker-side quota enforcement (quota.consumer.default) to apply as one budget across all streams. This prevents a single consumoor deployment from saturating broker NICs. --- pkg/consumoor/source/benthos.go | 4 ++++ pkg/consumoor/source/benthos_test.go | 36 ++++++++++++++++++++++++++++ pkg/consumoor/source/config.go | 5 ++++ 3 files changed, 45 insertions(+) diff --git a/pkg/consumoor/source/benthos.go b/pkg/consumoor/source/benthos.go index bf31cf7a..ce596601 100644 --- a/pkg/consumoor/source/benthos.go +++ b/pkg/consumoor/source/benthos.go @@ -138,6 +138,10 @@ func benthosConfigYAML(logLevel string, kafkaConfig *KafkaConfig) ([]byte, error "rebalance_timeout": kafkaConfig.RebalanceTimeout.String(), } + if kafkaConfig.ClientID != "" { + inputKafka["client_id"] = kafkaConfig.ClientID + } + if kafkaConfig.TopicRefreshInterval > 0 { inputKafka["metadata_max_age"] = kafkaConfig.TopicRefreshInterval.String() } diff --git a/pkg/consumoor/source/benthos_test.go b/pkg/consumoor/source/benthos_test.go index 6328759f..7a5e8650 100644 --- a/pkg/consumoor/source/benthos_test.go +++ b/pkg/consumoor/source/benthos_test.go @@ -219,6 +219,42 @@ func TestBenthosConfigYAML_NoTCPBlockWhenConnectTimeoutZero(t *testing.T) { _, hasTCP := kafka["tcp"] assert.False(t, hasTCP, "tcp block should be absent when ConnectTimeout is 0") + + _, hasClientID := kafka["client_id"] + assert.False(t, hasClientID, "client_id should be absent when ClientID is empty") +} + +func TestBenthosConfigYAML_ClientID(t *testing.T) { + cfg := &KafkaConfig{ + Brokers: []string{"kafka-1:9092"}, + Topics: []string{"^test-.+"}, + ConsumerGroup: "xatu-consumoor", + ClientID: "xatu-consumoor", + Encoding: "json", + FetchMinBytes: 1, + FetchWaitMaxMs: 250, + MaxPartitionFetchBytes: 1048576, + FetchMaxBytes: 10485760, + SessionTimeoutMs: 30000, + RebalanceTimeout: 15 * time.Second, + OffsetDefault: "earliest", + CommitInterval: 5 * time.Second, + ConnectTimeout: 10 * time.Second, + } + + raw, err := benthosConfigYAML("info", cfg) + require.NoError(t, err) + + var parsed map[string]any + require.NoError(t, yaml.Unmarshal(raw, &parsed)) + + input, ok := parsed["input"].(map[string]any) + require.True(t, ok) + + kafka, ok := input["kafka_franz"].(map[string]any) + require.True(t, ok) + + assert.Equal(t, "xatu-consumoor", kafka["client_id"]) } func TestBenthosSASLObjectUsesPasswordFile(t *testing.T) { diff --git a/pkg/consumoor/source/config.go b/pkg/consumoor/source/config.go index 5fc4d052..e2f88db6 100644 --- a/pkg/consumoor/source/config.go +++ b/pkg/consumoor/source/config.go @@ -59,6 +59,11 @@ type KafkaConfig struct { Topics []string `yaml:"topics"` // ConsumerGroup is the Kafka consumer group ID. ConsumerGroup string `yaml:"consumerGroup"` + // ClientID is the Kafka client ID sent to brokers. When set, all per-topic + // streams share this value, which allows broker-side quota enforcement + // (quota.consumer.default) to apply as a single budget across all streams. + // If empty, franz-go generates a unique ID per stream. + ClientID string `yaml:"clientId"` // Encoding is the message encoding format ("json" or "protobuf"). Encoding string `yaml:"encoding" default:"json"` From a648973df62c8822e37e9268339ab919f6848674 Mon Sep 17 00:00:00 2001 From: Andrew Davis <1709934+Savid@users.noreply.github.com> Date: Wed, 8 Apr 2026 12:44:56 +1000 Subject: [PATCH 27/27] consumoor: remove kzg_commitments array from data column sidecar table The kzg_commitments Array(FixedString(98)) column is redundant since kzg_commitments_count already captures the needed information. This removes the column from the ClickHouse migration and strips the corresponding field, serialization, reset, and snapshot logic from the beacon consumoor route. Co-Authored-By: Claude Opus 4.6 (1M context) --- deploy/migrations/clickhouse/001_init.up.sql | 1 - ...n_api_eth_v1_events_data_column_sidecar.gen.go | 15 +++------------ ...eacon_api_eth_v1_events_data_column_sidecar.go | 9 --------- 3 files changed, 3 insertions(+), 22 deletions(-) diff --git a/deploy/migrations/clickhouse/001_init.up.sql b/deploy/migrations/clickhouse/001_init.up.sql index ea57c032..5af77dd6 100644 --- a/deploy/migrations/clickhouse/001_init.up.sql +++ b/deploy/migrations/clickhouse/001_init.up.sql @@ -332,7 +332,6 @@ CREATE TABLE IF NOT EXISTS default.beacon_api_eth_v1_events_data_column_sidecar_ `block_root` FixedString(66) COMMENT 'The beacon block root hash in the beacon API event stream payload' CODEC(ZSTD(1)), `column_index` UInt64 COMMENT 'The index of column in the beacon API event stream payload' CODEC(ZSTD(1)), `kzg_commitments_count` UInt32 COMMENT 'Number of KZG commitments associated with the record' CODEC(ZSTD(1)), - `kzg_commitments` Array(FixedString(98)) COMMENT 'The KZG commitments in the beacon API event stream payload' CODEC(ZSTD(1)), `meta_client_name` LowCardinality(String) COMMENT 'Name of the client that generated the event', `meta_client_version` LowCardinality(String) COMMENT 'Version of the client that generated the event', `meta_client_implementation` LowCardinality(String) COMMENT 'Implementation of the client that generated the event', diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go index ed72fef2..27c951f8 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.gen.go @@ -23,7 +23,6 @@ type beaconApiEthV1EventsDataColumnSidecarBatch struct { BlockRoot route.SafeColFixedStr ColumnIndex proto.ColUInt64 KzgCommitmentsCount proto.ColUInt32 - KzgCommitments *proto.ColArr[[]byte] MetaClientName proto.ColStr MetaClientVersion proto.ColStr MetaClientImplementation proto.ColStr @@ -48,13 +47,8 @@ type beaconApiEthV1EventsDataColumnSidecarBatch struct { func newbeaconApiEthV1EventsDataColumnSidecarBatch() *beaconApiEthV1EventsDataColumnSidecarBatch { return &beaconApiEthV1EventsDataColumnSidecarBatch{ - EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), - BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), - KzgCommitments: func() *proto.ColArr[[]byte] { - var fs route.SafeColFixedStr - fs.SetSize(98) - return proto.NewArray[[]byte](&fs) - }(), + EventDateTime: func() proto.ColDateTime64 { var c proto.ColDateTime64; c.WithPrecision(proto.Precision(3)); return c }(), + BlockRoot: func() route.SafeColFixedStr { var c route.SafeColFixedStr; c.SetSize(66); return c }(), MetaClientIP: new(proto.ColIPv6).Nullable(), MetaClientGeoLongitude: new(proto.ColFloat64).Nullable(), MetaClientGeoLatitude: new(proto.ColFloat64).Nullable(), @@ -125,7 +119,6 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Input() proto.Input { {Name: "block_root", Data: &b.BlockRoot}, {Name: "column_index", Data: &b.ColumnIndex}, {Name: "kzg_commitments_count", Data: &b.KzgCommitmentsCount}, - {Name: "kzg_commitments", Data: b.KzgCommitments}, {Name: "meta_client_name", Data: &b.MetaClientName}, {Name: "meta_client_version", Data: &b.MetaClientVersion}, {Name: "meta_client_implementation", Data: &b.MetaClientImplementation}, @@ -159,7 +152,6 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Reset() { b.BlockRoot.Reset() b.ColumnIndex.Reset() b.KzgCommitmentsCount.Reset() - b.KzgCommitments.Reset() b.MetaClientName.Reset() b.MetaClientVersion.Reset() b.MetaClientImplementation.Reset() @@ -187,7 +179,7 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Snapshot() []map[string]any out := make([]map[string]any, n) for i := 0; i < n; i++ { - row := make(map[string]any, 30) + row := make(map[string]any, 29) row["updated_date_time"] = b.UpdatedDateTime.Row(i).Unix() row["event_date_time"] = b.EventDateTime.Row(i).UnixMilli() row["slot"] = b.Slot.Row(i) @@ -198,7 +190,6 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) Snapshot() []map[string]any row["block_root"] = string(b.BlockRoot.Row(i)) row["column_index"] = b.ColumnIndex.Row(i) row["kzg_commitments_count"] = b.KzgCommitmentsCount.Row(i) - row["kzg_commitments"] = route.ByteSlicesToStrings(b.KzgCommitments.Row(i)) row["meta_client_name"] = b.MetaClientName.Row(i) row["meta_client_version"] = b.MetaClientVersion.Row(i) row["meta_client_implementation"] = b.MetaClientImplementation.Row(i) diff --git a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go index 5b68ee3f..3fd7cf00 100644 --- a/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go +++ b/pkg/consumoor/route/beacon/beacon_api_eth_v1_events_data_column_sidecar.go @@ -104,15 +104,6 @@ func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendPayload(event *xatu.D } else { b.KzgCommitmentsCount.Append(0) } - - commitments := sidecar.GetKzgCommitments() //nolint:staticcheck // deprecated but still populated - byteSlices := make([][]byte, len(commitments)) - - for i, c := range commitments { - byteSlices[i] = []byte(c) - } - - b.KzgCommitments.Append(byteSlices) } func (b *beaconApiEthV1EventsDataColumnSidecarBatch) appendAdditionalData(